From 9b369689313f9ec48b24f9cb788c18650d3389cb Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 11 Jan 2016 14:44:05 -0800 Subject: [PATCH 01/16] Initial TaskTier commit --- extensions-core/tasktier/pom.xml | 104 ++ .../java/io/druid/cli/CliTierCreator.java | 36 + .../main/java/io/druid/cli/CliTierFork.java | 236 +++ .../main/java/io/druid/cli/CliTierLocal.java | 91 + .../druid/cli/ExecutorLifecycleProvider.java | 97 ++ .../main/java/io/druid/cli/ForkAnnouncer.java | 75 + .../AbstractTierRemoteTaskRunner.java | 866 ++++++++++ .../druid/indexing/overlord/PortWriter.java | 107 ++ .../overlord/PostingTierRemoteTaskRunner.java | 59 + .../overlord/TierLocalTaskRunner.java | 1539 +++++++++++++++++ .../overlord/TierLocalTaskRunnerFactory.java | 91 + .../overlord/TierRoutingTaskRunner.java | 207 +++ .../TierRoutingTaskRunnerFactory.java | 47 + .../indexing/overlord/TierTaskDiscovery.java | 131 ++ .../TierRoutingManagementStrategy.java | 245 +++ .../overlord/config/TierConfigModule.java | 55 + .../overlord/config/TierForkZkConfig.java | 74 + .../config/TierLocalTaskRunnerConfig.java | 112 ++ .../overlord/resources/DeadhandMonitor.java | 203 +++ .../overlord/resources/DeadhandResource.java | 89 + .../resources/ShutdownCleanlyResource.java | 58 + .../overlord/resources/TaskLogResource.java | 87 + .../resources/TierRunningCheckResource.java | 37 + .../routing/ForkingTaskRunnerTierFactory.java | 111 ++ .../routing/RemoteTaskRunnerTierFactory.java | 133 ++ ...askStatusPostToLeaderListenerResource.java | 77 + .../TaskStatusPostToLeaderReporter.java | 120 ++ .../overlord/routing/TaskStatusReporter.java | 44 + .../routing/TaskStatusReporterModule.java | 171 ++ .../overlord/routing/TierRouteConfig.java | 62 + .../routing/TierTaskRunnerFactory.java | 42 + .../overlord/routing/UnknownRouteFactory.java | 35 + .../services/io.druid.cli.CliCommandCreator | 20 + .../io.druid.initialization.DruidModule | 21 + .../java/io/druid/cli/CliTierCreatorTest.java | 55 + .../java/io/druid/cli/CliTierForkTest.java | 193 +++ .../java/io/druid/cli/CliTierLocalTest.java | 61 + .../cli/ExecutorLifecycleProviderTest.java | 229 +++ .../java/io/druid/cli/ForkAnnouncerTest.java | 77 + .../AbstractTierRemoteTaskRunnerTest.java | 1044 +++++++++++ .../io/druid/indexing/overlord/BusyTask.java | 109 ++ .../overlord/ForkingTaskRunnerTestModule.java | 92 + .../indexing/overlord/PortWriterTest.java | 171 ++ .../TierLocalTaskRunnerFactoryTest.java | 87 + .../overlord/TierLocalTaskRunnerTest.java | 572 ++++++ .../overlord/TierRoutingTaskRunnerTest.java | 308 ++++ .../overlord/TierTaskDiscoveryTest.java | 186 ++ .../TierRoutingManagementStrategyTest.java | 228 +++ .../resources/DeadhandMonitorTest.java | 143 ++ .../resources/DeadhandResourceTest.java | 303 ++++ .../resources/TaskLogResourceTest.java | 153 ++ .../TierRunningCheckResourceTest.java | 38 + .../routing/DynamicUpstreamReporterTest.java | 147 ++ .../RemoteTaskRunnerTierFactoryTest.java | 93 + ...tatusPostToLeaderListenerResourceTest.java | 111 ++ .../TaskStatusPostToLeaderReporterTest.java | 383 ++++ .../routing/TaskStatusReporterModuleTest.java | 48 + .../overlord/routing/TierRouteConfigTest.java | 90 + .../routing/UnknownRouteFactoryTest.java | 33 + .../io.druid.initialization.DruidModule | 19 + .../overlord/ForkingTaskRunnerFactory.java | 1 + .../overlord/autoscaling/ScalingStats.java | 8 + pom.xml | 50 +- .../main/java/io/druid/cli/CliOverlord.java | 2 +- 64 files changed, 10514 insertions(+), 2 deletions(-) create mode 100644 extensions-core/tasktier/pom.xml create mode 100644 extensions-core/tasktier/src/main/java/io/druid/cli/CliTierCreator.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/cli/CliTierLocal.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/cli/ForkAnnouncer.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunner.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PortWriter.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PostingTierRemoteTaskRunner.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunnerFactory.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierTaskDiscovery.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierForkZkConfig.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierLocalTaskRunnerConfig.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandMonitor.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandResource.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/ShutdownCleanlyResource.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TaskLogResource.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TierRunningCheckResource.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/ForkingTaskRunnerTierFactory.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactory.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResource.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporter.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporterModule.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierRouteConfig.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierTaskRunnerFactory.java create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/UnknownRouteFactory.java create mode 100644 extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.cli.CliCommandCreator create mode 100644 extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.initialization.DruidModule create mode 100644 extensions-core/tasktier/src/test/java/io/druid/cli/CliTierCreatorTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/cli/CliTierLocalTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/cli/ExecutorLifecycleProviderTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/cli/ForkAnnouncerTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunnerTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/BusyTask.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/ForkingTaskRunnerTestModule.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/PortWriterTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierLocalTaskRunnerFactoryTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierLocalTaskRunnerTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierRoutingTaskRunnerTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierTaskDiscoveryTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandMonitorTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandResourceTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TierRunningCheckResourceTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResourceTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporterTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusReporterModuleTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TierRouteConfigTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/UnknownRouteFactoryTest.java create mode 100644 extensions-core/tasktier/src/test/resources/META-INF/services/io.druid.initialization.DruidModule diff --git a/extensions-core/tasktier/pom.xml b/extensions-core/tasktier/pom.xml new file mode 100644 index 000000000000..732b71fc55bb --- /dev/null +++ b/extensions-core/tasktier/pom.xml @@ -0,0 +1,104 @@ + + + + 4.0.0 + + io.druid.extensions + druid-tasktier + druid-tasktier + druid-tasktier + + + io.druid + druid + 0.9.1-SNAPSHOT + ../../pom.xml + + + + + io.druid + druid-processing + provided + + + + io.druid + druid-services + provided + + + + io.druid + druid-indexing-service + provided + + + + io.druid + druid-server + provided + + + + com.sun + tools + 1.8 + system + ${java.home}/../lib/tools.jar + + + + + io.druid + druid-processing + ${project.parent.version} + test + test-jar + + + junit + junit + test + + + org.easymock + easymock + test + + + org.apache.curator + curator-test + test + + + io.druid + druid-server + ${project.version} + test + test-jar + + + com.carrotsearch + junit-benchmarks + test + + + + diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierCreator.java b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierCreator.java new file mode 100644 index 000000000000..f723921a0eea --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierCreator.java @@ -0,0 +1,36 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import io.airlift.airline.Cli; + + +public class CliTierCreator implements CliCommandCreator +{ + @Override + public void addCommands(Cli.CliBuilder cliBuilder) + { + cliBuilder + .withGroup("tier") + .withCommands(CliTierFork.class, CliTierLocal.class) + .withDefaultCommand(CliTierFork.class) + .withDescription("Run a tier for executing tasks"); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java new file mode 100644 index 000000000000..57767f583e51 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java @@ -0,0 +1,236 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provider; +import com.google.inject.Provides; +import com.metamx.common.logger.Logger; +import io.airlift.airline.Command; +import io.druid.guice.Jerseys; +import io.druid.guice.LazySingleton; +import io.druid.guice.LifecycleModule; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.ManageLifecycleLast; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.RemoteChatHandler; +import io.druid.guice.annotations.Self; +import io.druid.indexing.common.actions.TaskActionClientFactory; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.PortWriter; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; +import io.druid.indexing.overlord.resources.DeadhandMonitor; +import io.druid.indexing.overlord.resources.DeadhandResource; +import io.druid.indexing.overlord.resources.ShutdownCleanlyResource; +import io.druid.indexing.overlord.resources.TaskLogResource; +import io.druid.indexing.overlord.resources.TierRunningCheckResource; +import io.druid.indexing.worker.executor.ExecutorLifecycle; +import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; +import io.druid.server.DruidNode; +import io.druid.server.initialization.ServerConfig; +import io.druid.server.initialization.jetty.ChatHandlerServerModule; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.TimeoutException; + +@Command( + name = "fork", + description = "Runs a task as part of a tier. This command forks / joins the task in the same JVM. " + + "It expects the current working path to contain the files of interest" +) +public class CliTierFork extends CliPeon +{ + private static final Logger LOG = new Logger(CliTierFork.class); + + @Override + protected List getModules() + { + final List modules = new ArrayList<>(super.getModules()); + final Iterator moduleIterator = modules.iterator(); + while (moduleIterator.hasNext()) { + if (moduleIterator.next() instanceof ChatHandlerServerModule) { + // ChatHandlerServerModule injection is all screwed up + moduleIterator.remove(); + break; + } + } + modules.add( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(Task.class).toProvider(TaskProvider.class).in(LazySingleton.class); + + binder.bind(PortWriter.class).in(ManageLifecycleLast.class); + LifecycleModule.register(binder, PortWriter.class); + + binder.bind(ParentMonitorInputStreamFaker.class) + .toProvider(ParentMonitorInputStreamFakerProvider.class) + .in(LazySingleton.class); + + Jerseys.addResource(binder, ShutdownCleanlyResource.class); + Jerseys.addResource(binder, DeadhandResource.class); + Jerseys.addResource(binder, TierRunningCheckResource.class); + Jerseys.addResource(binder, TaskLogResource.class); + + binder.bind(DeadhandMonitor.class).in(ManageLifecycle.class); + LifecycleModule.register(binder, DeadhandMonitor.class); + + binder.bind(ForkAnnouncer.class).in(ManageLifecycle.class); + LifecycleModule.register(binder, ForkAnnouncer.class); + + // Chat handler not used + binder.bind(DruidNode.class) + .annotatedWith(RemoteChatHandler.class) + .to(Key.get(DruidNode.class, Self.class)); + binder.bind(ServerConfig.class).annotatedWith(RemoteChatHandler.class).to(Key.get(ServerConfig.class)); + } + } + ); + return modules; + } + + @Provides + @ManageLifecycle + public static ExecutorLifecycle executorLifecycleProvider( + final TaskActionClientFactory taskActionClientFactory, + final TaskRunner taskRunner, + final TaskConfig taskConfig, + final ParentMonitorInputStreamFaker parentStream, + final @Json ObjectMapper jsonMapper, + final ExecutorLifecycleConfig config + ) + { + final File taskFile = config.getTaskFile(); + final File statusFile = config.getStatusFile(); + try { + if (!statusFile.exists() && !statusFile.createNewFile()) { + throw new IOException(String.format("Could not create file [%s]", statusFile)); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + LOG.info("Using status and task files: [%s] [%s]", statusFile, taskFile); + return new ExecutorLifecycle( + new ExecutorLifecycleConfig() + { + // This stream closes whenever the parent dies. + @Override + public InputStream getParentStream() + { + return parentStream; + } + } + .setStatusFile(statusFile) + .setTaskFile(taskFile), + taskConfig, + taskActionClientFactory, + taskRunner, + jsonMapper + ); + } +} + +class ParentMonitorInputStreamFakerProvider implements Provider +{ + final DeadhandResource deadhandResource; + final TierLocalTaskRunnerConfig tierLocalTaskRunnerConfig; + + @Inject + ParentMonitorInputStreamFakerProvider( + final DeadhandResource deadhandResource, + final TierLocalTaskRunnerConfig tierLocalTaskRunnerConfig + ) + { + this.deadhandResource = deadhandResource; + this.tierLocalTaskRunnerConfig = tierLocalTaskRunnerConfig; + } + + @Override + public ParentMonitorInputStreamFaker get() + { + return new ParentMonitorInputStreamFaker() + { + @Override + public int read() throws IOException + { + try { + deadhandResource.waitForHeartbeat(tierLocalTaskRunnerConfig.getHeartbeatTimeLimit()); + return 0; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } + catch (TimeoutException e) { + // fake EOS + return -1; + } + } + }; + } +} + +abstract class ParentMonitorInputStreamFaker extends InputStream +{ + +} + +class TaskProvider implements Provider +{ + private final ObjectMapper jsonMapper; + private final File taskJson; + + @Inject + public TaskProvider( + @Json + ObjectMapper mapper, + ExecutorLifecycleConfig config + ) + { + jsonMapper = mapper; + this.taskJson = config.getTaskFile(); + } + + @Override + public Task get() + { + try { + return jsonMapper.readValue(taskJson, Task.class); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierLocal.java b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierLocal.java new file mode 100644 index 000000000000..d378a041e539 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierLocal.java @@ -0,0 +1,91 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.util.Modules; +import com.metamx.common.logger.Logger; +import io.airlift.airline.Command; +import io.druid.guice.LazySingleton; +import io.druid.guice.ManageLifecycle; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TierLocalTaskRunner; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import io.druid.indexing.worker.Worker; +import io.druid.indexing.worker.WorkerCuratorCoordinator; +import io.druid.server.initialization.IndexerZkConfig; +import org.apache.curator.framework.CuratorFramework; + +import java.util.List; + +@Command( + name = "local", + description = "Runs a task as part of a tier. This command creates and monitors a 'fork' tier task locally" +) +public class CliTierLocal extends CliMiddleManager +{ + private static final Logger LOG = new Logger(CliTierLocal.class); + + @Override + protected List getModules() + { + return ImmutableList.of(Modules.override(super.getModules()).with( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(TaskRunner.class).to(TierLocalTaskRunner.class); + binder.bind(TierLocalTaskRunner.class).in(LazySingleton.class); + } + + @Provides + @ManageLifecycle + public WorkerCuratorCoordinator disabledWorkerCuratorCoordinator( + ObjectMapper jsonMapper, + IndexerZkConfig indexerZkConfig, + RemoteTaskRunnerConfig config, + CuratorFramework curatorFramework, + Worker worker + ) + { + return new WorkerCuratorCoordinator(jsonMapper, indexerZkConfig, config, curatorFramework, worker) + { + @Override + public void start() throws Exception + { + LOG.debug("WorkerCuratorCoordinator Start NOOP"); + } + + @Override + public void stop() throws Exception + { + LOG.debug("WorkerCuratorCoordinator Stop NOOP"); + } + }; + } + } + )); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java b/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java new file mode 100644 index 000000000000..1f46446220c5 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java @@ -0,0 +1,97 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.google.inject.Provider; +import com.metamx.common.logger.Logger; +import io.druid.guice.annotations.Json; +import io.druid.indexing.common.actions.TaskActionClientFactory; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TierLocalTaskRunner; +import io.druid.indexing.worker.executor.ExecutorLifecycle; +import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Paths; + +public class ExecutorLifecycleProvider implements Provider +{ + private static final Logger log = new Logger(ExecutorLifecycleProvider.class); + private final TaskActionClientFactory taskActionClientFactory; + private final TaskRunner taskRunner; + private final TaskConfig taskConfig; + private final InputStream parentStream; + private final ObjectMapper jsonMapper; + + @Inject + public ExecutorLifecycleProvider( + final TaskActionClientFactory taskActionClientFactory, + final TaskRunner taskRunner, + final TaskConfig taskConfig, + final ParentMonitorInputStreamFaker parentStream, + final @Json ObjectMapper jsonMapper + ) + { + this.taskActionClientFactory = taskActionClientFactory; + this.taskRunner = taskRunner; + this.taskConfig = taskConfig; + this.parentStream = parentStream; + this.jsonMapper = jsonMapper; + } + + @Override + public ExecutorLifecycle get() + { + final File statusFile = Paths.get(TierLocalTaskRunner.STATUS_FILE_NAME).toFile(); + final File taskFile = Paths.get(TierLocalTaskRunner.TASK_FILE_NAME).toFile(); + try { + if (!statusFile.exists() && !statusFile.createNewFile()) { + throw new IOException(String.format("Could not create file [%s]", statusFile)); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + log.info("Using status and task files: [%s] [%s]", statusFile, taskFile); + return new ExecutorLifecycle( + new ExecutorLifecycleConfig() + { + // This stream closes whenever the parent dies. + @Override + public InputStream getParentStream() + { + return parentStream; + } + } + .setStatusFile(statusFile) + .setTaskFile(taskFile), + taskConfig, + taskActionClientFactory, + taskRunner, + jsonMapper + ); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/ForkAnnouncer.java b/extensions-core/tasktier/src/main/java/io/druid/cli/ForkAnnouncer.java new file mode 100644 index 000000000000..27b903029ca3 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/ForkAnnouncer.java @@ -0,0 +1,75 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import io.druid.curator.announcement.Announcer; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.Self; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.config.TierForkZkConfig; +import io.druid.server.DruidNode; + +import java.io.IOException; + +// Announces that it has a task +public class ForkAnnouncer +{ + private final Announcer announcer; + private final DruidNode druidNode; + private final ObjectMapper jsonMapper; + private final String path; + + @Inject + public ForkAnnouncer( + Announcer announcer, + @Self DruidNode self, + @Json ObjectMapper mapper, + TierForkZkConfig tierForkZkConfig, + Task task + ) + { + this.announcer = announcer; + druidNode = self; + jsonMapper = mapper; + path = tierForkZkConfig.getTierTaskIDPath(task.getId()); + } + + @LifecycleStart + public void announceTask() + { + try { + announcer.announce(path, jsonMapper.writeValueAsBytes(druidNode)); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + @LifecycleStop + public void unannounceTask() + { + announcer.unannounce(path); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunner.java new file mode 100644 index 000000000000..ebd022d5e430 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunner.java @@ -0,0 +1,866 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.base.Optional; +import com.google.common.base.Predicate; +import com.google.common.base.Throwables; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import com.google.common.io.ByteSource; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListenableScheduledFuture; +import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.Pair; +import com.metamx.common.RetryUtils; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.emitter.EmittingLogger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.response.ClientResponse; +import com.metamx.http.client.response.HttpResponseHandler; +import com.metamx.http.client.response.InputStreamResponseHandler; +import io.druid.guice.annotations.Global; +import io.druid.indexing.common.TaskLocation; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.autoscaling.ScalingStats; +import io.druid.indexing.overlord.resources.ShutdownCleanlyResource; +import io.druid.indexing.overlord.resources.TaskLogResource; +import io.druid.indexing.overlord.resources.TierRunningCheckResource; +import io.druid.indexing.overlord.routing.TaskStatusReporter; +import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogStreamer; +import org.jboss.netty.handler.codec.http.HttpChunk; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.DateTime; +import org.joda.time.Duration; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.InputStream; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Should only be instanced on the overlord. Instancing it anywhere else requires TaskStorage to be functioning from that + * node + */ +public abstract class AbstractTierRemoteTaskRunner implements TaskRunner, TaskLogStreamer, TaskStatusReporter +{ + private static final EmittingLogger log = new EmittingLogger(AbstractTierRemoteTaskRunner.class); + private final ConcurrentMap knownTasks = new ConcurrentHashMap<>(); + private final TierTaskDiscovery tierTaskDiscovery; + private final HttpClient httpClient; + private final TaskStorage taskStorage; + private final ListeningScheduledExecutorService cleanupExec; + private final ListenableScheduledFuture cleanupFuture; + private final AtomicBoolean started = new AtomicBoolean(false); + + @Inject + public AbstractTierRemoteTaskRunner( + TierTaskDiscovery tierTaskDiscovery, + @Global HttpClient httpClient, + TaskStorage taskStorage, + ScheduledExecutorFactory executorFactory + ) + { + this.cleanupExec = MoreExecutors.listeningDecorator(executorFactory.create(1, "AbstractTierRemoteTaskCleanup--%s")); + this.tierTaskDiscovery = tierTaskDiscovery; + this.httpClient = httpClient; + this.taskStorage = taskStorage; + cleanupFuture = cleanupExec.scheduleAtFixedRate( + new Runnable() + { + @Override + public void run() + { + try { + synchronized (started) { + if (!started.get()) { + log.debug("Not yet started"); + return; + } + for (StatefulTaskRunnerWorkItem workItem : knownTasks.values()) { + if (workItem.lastSeen.get() > 300_000 // TODO: configurable + && StatefulTaskRunnerWorkItem.State.isTerminal(workItem.state.get())) { + // If its done and has not been updated in 5 minutes, remove it from the tasks + if (!knownTasks.remove(workItem.getTaskId(), workItem)) { + log.makeAlert("Task [%s] could not be removed", workItem.getTaskId()).emit(); + } + } + } + } + } + catch (Exception e) { + log.error(e, "Unhandled exception in cleanup"); + } + } + }, + 0, // launch and probably skip once. Useful for unit tests + 1, + TimeUnit.MINUTES + ); + Futures.addCallback(cleanupFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + log.debug("Cleanup future is done"); + } + + @Override + public void onFailure(Throwable t) + { + if (cleanupFuture.isCancelled()) { + log.debug("Cleanup is cancelled"); + } else { + log.error(t, "Cleanup had an error"); + } + } + }); + } + + @Override + public Optional streamTaskLog(String taskid, long offset) throws IOException + { + final StatefulTaskRunnerWorkItem workItem = knownTasks.get(taskid); + DruidNode node = workItem == null ? null : workItem.node.get(); + if (node == null) { + final Optional maybeNode = tierTaskDiscovery.getNodeForTask(taskid); + if (!maybeNode.isPresent()) { + log.debug("No node discovered for task id [%s]", taskid); + return Optional.absent(); + } + node = maybeNode.get(); + } + if (node == null) { + log.debug("No node for task id [%s]", taskid); + return Optional.absent(); + } + final URL url = TaskLogResource.buildURL(node, offset); + return Optional.of( + new ByteSource() + { + @Override + public InputStream openStream() throws IOException + { + try { + return httpClient.go( + new Request(HttpMethod.GET, url), + new InputStreamResponseHandler(), + Duration.millis(120_000) // TODO: make this configurable + ).get(); + } + catch (InterruptedException e) { + log.info(e, "Interrupted while fetching logs from [%s]", url); + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + log.warn(e, "Error getting data from [%s]", url); + final Throwable cause = e.getCause(); + Throwables.propagateIfInstanceOf(cause, IOException.class); + throw Throwables.propagate(cause); + } + } + } + ); + } + + @Override + public List>> restore() + { + return ImmutableList.of(); + } + + /** + * Handle task state in the process of running it. The actual run request is handled through the `launch` method + * + * @param task task to run + * + * @return A future which will return the value TaskStatus. Any exceptions encountered during `launch` will be exposed + * as exceptions in the future, including if the launch was interrupted by InterruptedException. + */ + @Override + public ListenableFuture run(final Task task) + { + final SettableFuture future = SettableFuture.create(); + Futures.addCallback( + future, new FutureCallback() + { + @Override + public void onSuccess(TaskStatus status) + { + final StatefulTaskRunnerWorkItem priorItem = knownTasks.get(task.getId()); + if (priorItem == null) { + log.makeAlert("Task [%s] was pruned before it could be reported with status [%s]", task.getId(), status) + .emit(); + return; + } + priorItem.seen(); + if (status.isSuccess()) { + priorItem.state.set(StatefulTaskRunnerWorkItem.State.SUCCESS); + } else if (status.isFailure()) { + priorItem.state.set(StatefulTaskRunnerWorkItem.State.FAILED); + } + } + + @Override + public void onFailure(Throwable throwable) + { + if (future.isCancelled()) { + log.info("Future for [%s] was cancelled", task.getId()); + return; + } + final StatefulTaskRunnerWorkItem priorItem = knownTasks.get(task.getId()); + if (priorItem == null) { + log.warn("Task [%s] was pruned before it could be reported as unknown", task.getId()); + return; + } + priorItem.seen(); + final StatefulTaskRunnerWorkItem.State state = priorItem.state.get(); + if (StatefulTaskRunnerWorkItem.State.FAILED.equals(state) + || StatefulTaskRunnerWorkItem.State.SUCCESS.equals(state)) { + log.debug("Task [%s] already done, skipping setting state from failed future", task.getId()); + } else { + log.debug("Setting state to UNKNOWN for failed future task [%s]", task.getId()); + if (!priorItem.state.compareAndSet(state, StatefulTaskRunnerWorkItem.State.UNKNOWN)) { + log.warn( + "state for task [%s] was changed during failure update to UNKNOWN. Expected [%s] found [%s]", + task.getId(), + state, + priorItem.state.get() + ); + } + } + } + } + ); + final StatefulTaskRunnerWorkItem workItem = new StatefulTaskRunnerWorkItem( + task.getId(), + future, + DateTime.now(), + DateTime.now() + ); + workItem.state.set(StatefulTaskRunnerWorkItem.State.STAGED); + final StatefulTaskRunnerWorkItem priorItem = knownTasks.putIfAbsent(task.getId(), workItem); + if (priorItem != null) { + return priorItem.getResult(); + } + try { + launch(future, task); + } + catch (Exception e) { + if (!future.isDone()) { + future.setException(e); + } else { + log.warn(e, "Task [%s] future already done, ignoring error"); + } + } + return future; + } + + /** + * Launch the task on the framework + * + * @param future The future which will be used + * @param task The task to launch + */ + protected abstract void launch(SettableFuture future, Task task); + + /** + * Kill the task. May be overridden by any particular implementation. + * + * @param node The DruidNode (host:port) of the druid JVM running a TASK. + */ + protected ListenableFuture kill(DruidNode node) + { + return killRemoteTask(httpClient, node); + } + + @Override + public void shutdown(String taskid) + { + StatefulTaskRunnerWorkItem workItem = knownTasks.get(taskid); + if (workItem == null) { + refreshTaskIds(); + final SettableFuture future = SettableFuture.create(); + future.set(TaskStatus.failure(taskid)); + final StatefulTaskRunnerWorkItem statefulTaskRunnerWorkItem = new StatefulTaskRunnerWorkItem( + taskid, + future, + DateTime.now(), + DateTime.now() + ); + workItem = knownTasks.putIfAbsent(taskid, statefulTaskRunnerWorkItem); + if (workItem == null) { + log.info("Task [%s] not found, setting work item as failed"); + return; + } + } + + final DruidNode node = workItem.node.get(); + + if (node == null) { + log.error("Task [%s] has a work item but is not reporting itself, Failing task. Will kill if it reports itself"); + ((SettableFuture) workItem.getResult()).set(TaskStatus.failure(taskid)); + return; + } + try { + kill(node).get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + } + + @Override + public Collection getRunningTasks() + { + return Collections2.filter( + knownTasks.values(), + new Predicate() + { + @Override + public boolean apply(StatefulTaskRunnerWorkItem input) + { + return StatefulTaskRunnerWorkItem.State.STARTED.equals(input.state.get()); + } + } + ); + } + + @Override + public Collection getPendingTasks() + { + return Collections2.filter( + knownTasks.values(), + new Predicate() + { + @Override + public boolean apply(StatefulTaskRunnerWorkItem input) + { + return StatefulTaskRunnerWorkItem.State.STAGED.equals(input.state.get()); + } + } + ); + } + + @Override + public Collection getKnownTasks() + { + return knownTasks.values(); + } + + @Override + public Optional getScalingStats() + { + return Optional.absent(); + } + + @Override + public boolean reportStatus(final TaskStatus status) + { + boolean checkStorageStatus = false; // If we expect this status to match the final version in storage + final String taskId = status.getId(); + StatefulTaskRunnerWorkItem workItem = knownTasks.get(taskId); + if (workItem == null && status.isComplete()) { + final SettableFuture future = SettableFuture.create(); + if (!future.set(status)) { + throw new ISE("Somehow someone set the future first"); + } + workItem = new StatefulTaskRunnerWorkItem(taskId, future, DateTime.now(), DateTime.now()); + workItem = knownTasks.putIfAbsent(taskId, workItem); + if (workItem == null) { + checkStorageStatus = true; + } + // In the weird case where workItem encounters a race condition and is set, we will treat it properly below + } + + // Either workItem was present to begin with or it was added in a racy way + if (workItem != null) { + workItem.seen(); + final StatefulTaskRunnerWorkItem.State workItemState = workItem.state.get(); + if (!workItemState.isTerminal()) { + StatefulTaskRunnerWorkItem.State newState = StatefulTaskRunnerWorkItem.State.of(status.getStatusCode()); + if (workItem.state.compareAndSet(workItemState, newState)) { + log.debug("Set task [%s] to state [%s]", taskId, newState); + } else { + log.warn( + "Update of task [%s] to state [%s] failed because it was in state [%s]", + taskId, + newState, + workItem.state.get() + ); // Example : terminal status is reported in a racy way + } + } + } + if (workItem != null && status.isComplete()) { + final ListenableFuture workFuture = workItem.getResult(); + if (workFuture instanceof SettableFuture) { + final SettableFuture settableFuture = (SettableFuture) workFuture; + /** + * This is actually a bit dangerous. We're relying on the callbacks in TaskQueue to handle the future, + * but don't make any guarantees the future callbacks fire or complete as expected. As such it is possible + * to kill the overlord between the time the future is set and the result is properly stored in the task + * storage. We check at the end to make sure the status is expected. + */ + if (!settableFuture.set(status)) { + final TaskStatus priorStatus; + try { + priorStatus = settableFuture.get(); + } + catch (InterruptedException | ExecutionException e) { + throw new ISE(e, "How did this happen!? Expected the result for task [%s] to already be set", taskId); + } + if (!status.equals(priorStatus)) { + // Alert here or just error log? + log.error("Received task status of [%s] but already set to [%s]", status, priorStatus); + } else { + checkStorageStatus = true; + } + if (priorStatus.isFailure()) { + log.info("Already failed task reported status of [%s]", taskId); + } + } else { + log.debug("Task status for [%s] could not be set. Checking storage value", taskId); + checkStorageStatus = true; + } + } else { + throw new ISE("Future for task ID [%s] is not a SettableFuture: [%s]", taskId, workItem.getResult()); + } + } + + if (!checkStorageStatus) { + // We heard the report, but it didn't match and we don't care. We logged it. + return true; + } + + try { + // TODO: configurable + final AtomicInteger retriesRemaining = new AtomicInteger(10); + // We use RetryUtils for exponential backoff, but don't want it to throw ISE when retries are exceeded. + return RetryUtils.retry( + new Callable() + { + @Override + public Boolean call() throws Exception + { + if (retriesRemaining.decrementAndGet() <= 0) { + return false; + } + final Optional setStatus = taskStorage.getStatus(taskId); + if (setStatus.isPresent()) { + final TaskStatus setTaskStatus = setStatus.get(); + if (setTaskStatus.getStatusCode().equals(status.getStatusCode())) { + return true; + } + if (setTaskStatus.isComplete()) { + log.makeAlert( + "Task [%s] tried to set terminal status [%s] but was already [%s]", + taskId, + status, + setTaskStatus + ).emit(); + return true; + } else { + // Storage of task did not match and was not terminal in storage. + return false; + } + } else { + throw new NotYetFoundException(); + } + } + }, + new Predicate() + { + @Override + public boolean apply(Throwable throwable) + { + return throwable instanceof NotYetFoundException; + } + }, + retriesRemaining.get() + 1 + ); + } + catch (InterruptedException e) { + log.debug(e, "Reporting Interrupted"); + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (Exception e) { + log.error(e, "Error reporting status [%s]", status); + throw Throwables.propagate(e); + } + } + + protected void refreshTaskIds() + { + final Map taskMap = tierTaskDiscovery.getTasks(); + final Set lostTasks = Sets.difference(knownTasks.keySet(), taskMap.keySet()); + for (String lostTask : lostTasks) { + final StatefulTaskRunnerWorkItem workItem = knownTasks.get(lostTask); + if (workItem != null) { + final StatefulTaskRunnerWorkItem.State state = workItem.state.get(); + if (StatefulTaskRunnerWorkItem.State.STARTED.equals(state)) { + log.warn("Lost task [%s]", lostTask); + if (!workItem.state.compareAndSet(state, StatefulTaskRunnerWorkItem.State.UNKNOWN)) { + log.warn("Could not update task [%s] to UNKNOWN because it is [%s]", lostTask, workItem.state.get()); + } + } + } + } + for (Map.Entry entry : taskMap.entrySet()) { + StatefulTaskRunnerWorkItem workItem = knownTasks.get(entry.getKey()); + if (workItem == null) { + // Maybe put in a new entry + final SettableFuture future = SettableFuture.create(); + workItem = new StatefulTaskRunnerWorkItem(entry.getKey(), future, null, null); + final StatefulTaskRunnerWorkItem prior = knownTasks.putIfAbsent(entry.getKey(), workItem); + if (prior != null) { + log.debug("Found prior work item for task id [%s]", prior.getTaskId()); + workItem = prior; + } + } + final StatefulTaskRunnerWorkItem.State state = workItem.state.get(); + if (StatefulTaskRunnerWorkItem.State.STAGED.equals(state) + || StatefulTaskRunnerWorkItem.State.UNKNOWN.equals(state)) { + if (!workItem.state.compareAndSet(state, StatefulTaskRunnerWorkItem.State.STARTED)) { + log.warn( + "Someone set state for task [%s] to [%s] before I could set it to STARTED from [%s]", + entry.getKey(), + workItem.state.get(), + state + ); + } + } else if (!StatefulTaskRunnerWorkItem.State.STARTED.equals(state)) { + log.error("Task [%s] is reporting strange state [%s]", entry.getKey(), state); + } + workItem.node.set(entry.getValue()); + workItem.seen(); + } + } + + protected ListenableFuture refreshTaskStatus() + { + final List> futures = new ArrayList<>(); + for (final Map.Entry entry : knownTasks.entrySet()) { + final StatefulTaskRunnerWorkItem workItem = entry.getValue(); + final URL url; + try { + final DruidNode node = workItem.node.get(); + if (node == null) { + log.warn("Task [%s] has no node", entry.getKey()); + continue; + } + url = new URL("http", node.getHost(), node.getPort(), TierRunningCheckResource.PATH); + } + catch (MalformedURLException e) { + log.warn(e, "Error checking task [%s]", entry.getKey()); + final SettableFuture future = SettableFuture.create(); + future.setException(e); + futures.add(future); + continue; + } + final Request request = new Request(HttpMethod.GET, url); + final ListenableFuture future = httpClient.go( + request, + new HttpResponseHandler() + { + @Override + public ClientResponse handleResponse(HttpResponse response) + { + if (response.getStatus().getCode() == HttpResponseStatus.OK.getCode()) { + return ClientResponse.finished(true); + } else { + log.warn("Error in checking for status of task [%s]: [%s]", entry.getKey(), response.getStatus()); + return ClientResponse.finished(false); + } + } + + @Override + public ClientResponse handleChunk( + ClientResponse clientResponse, HttpChunk chunk + ) + { + log.trace("Received a chunk from [%s]", url); + return clientResponse; + } + + @Override + public ClientResponse done(ClientResponse clientResponse) + { + final StatefulTaskRunnerWorkItem.State state = workItem.state.get(); + if (state.isTerminal()) { + // Maybe it is shutting down? + log.warn("Task [%s] still being polled even though it is thought to be [%s]", entry.getKey(), state); + } else if (clientResponse.getObj()) { + log.debug("Task [%s] running", entry.getKey()); + // Example: failure is reported asynchronously + if (!workItem.state.compareAndSet(state, StatefulTaskRunnerWorkItem.State.STARTED)) { + log.warn( + "Failed to update STARTED state of task [%s]. Expected [%s] found [%s]", + entry.getKey(), + state, + workItem.state.get() + ); + } + } else { + // Example: failure is reported asynchronously + if (!workItem.state.compareAndSet(state, StatefulTaskRunnerWorkItem.State.UNKNOWN)) { + log.warn( + "Failed to update UNKNOWN state of task [%s]. Expected [%s] found [%s]", + entry.getKey(), + state, + workItem.state.get() + ); + } + } + return clientResponse; + } + + @Override + public void exceptionCaught(ClientResponse clientResponse, Throwable e) + { + final StatefulTaskRunnerWorkItem.State state = entry.getValue().state.get(); + if (StatefulTaskRunnerWorkItem.State.STARTED.equals(state)) { + log.error(e, "Error in processing [%s] setting [%s] to unknown", url, entry.getKey()); + if (!entry.getValue().state.compareAndSet(state, StatefulTaskRunnerWorkItem.State.UNKNOWN)) { + log.warn( + "Task [%s] could not update to UNKNOWN. Expected [%s] found [%s]", + entry.getKey(), + state, + entry.getValue().state.get() + ); + } + } else { + log.error(e, "Error processing [%s], leaving [%s] as [%s]", url, entry.getKey(), state); + } + } + } + ); + Futures.addCallback( + future, new FutureCallback() + { + @Override + public void onSuccess(Boolean result) + { + log.debug("[%s] resulted in [%s]", url, result); + } + + @Override + public void onFailure(Throwable t) + { + log.error(t, "Error getting status from [%s]", url); + } + } + ); + futures.add(future); + } + return Futures.allAsList(futures); + } + + public static ListenableFuture killRemoteTask(HttpClient httpClient, DruidNode node) + { + final URL url; + try { + url = new URL("http", node.getHost(), node.getPort(), ShutdownCleanlyResource.PATH); + } + catch (MalformedURLException e) { + throw Throwables.propagate(e); + } + return httpClient.go( + new Request(HttpMethod.POST, url), + new HttpResponseHandler() + { + @Override + public ClientResponse handleResponse(HttpResponse response) + { + HttpResponseStatus status = response.getStatus(); + log.debug( + "Received status code %d [%s] for shutdown request", + status.getCode(), + status.getReasonPhrase() + ); + if (status.getCode() != HttpResponseStatus.ACCEPTED.getCode()) { + final String msg = String.format( + "Bad status code. Received [%d]:[%s] from url [%s]", + status.getCode(), + status.getReasonPhrase(), + url + ); + throw new RuntimeException(msg); + } + return ClientResponse.finished(null); + } + + @Override + public ClientResponse handleChunk( + ClientResponse clientResponse, HttpChunk chunk + ) + { + log.debug("Received chunk... why?"); + return clientResponse; + } + + @Override + public ClientResponse done(ClientResponse clientResponse) + { + return clientResponse; + } + + @Override + public void exceptionCaught(ClientResponse clientResponse, Throwable e) + { + log.error(e, "Error in command execution"); + } + } + ); + } + + @Override + @LifecycleStart + public void start() + { + synchronized (started) { + if (!started.compareAndSet(false, true)) { + throw new ISE("Already started"); + } + refreshTaskIds(); + refreshTaskStatus(); + } + } + + @Override + @LifecycleStop + public void stop() + { + synchronized (started) { + started.set(false); + knownTasks.clear(); + cleanupExec.shutdownNow(); + } + } + + + @Override + public void registerListener(TaskRunnerListener listener, Executor executor) + { + throw new UnsupportedOperationException("TODO"); + } + + @Override + public void unregisterListener(String listenerId) + { + throw new UnsupportedOperationException("Not Yet Supported"); + } +} + +class StatefulTaskRunnerWorkItem extends TaskRunnerWorkItem +{ + final AtomicReference state = new AtomicReference<>(State.UNKNOWN); + final AtomicLong lastSeen = new AtomicLong(System.currentTimeMillis()); + final AtomicReference node = new AtomicReference<>(null); + + public StatefulTaskRunnerWorkItem( + String taskId, + SettableFuture result, + DateTime createdTime, + DateTime queueInsertionTime + ) + { + super(taskId, result, createdTime, queueInsertionTime); + } + + @Override + public TaskLocation getLocation() + { + throw new UnsupportedOperationException("TODO: this"); + } + + enum State + { + UNKNOWN, STAGED, STARTED, FAILED, SUCCESS; + + public static boolean isTerminal(State state) + { + return SUCCESS.equals(state) || FAILED.equals(state); + } + + public static State of(TaskStatus.Status status) + { + switch (status) { + case RUNNING: + return STARTED; + case SUCCESS: + return SUCCESS; + case FAILED: + return FAILED; + default: + return UNKNOWN; + } + } + + public boolean isTerminal() + { + return isTerminal(this); + } + } + + public void seen() + { + lastSeen.set(System.currentTimeMillis()); + } +} + +class NotYetFoundException extends Exception +{ +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PortWriter.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PortWriter.java new file mode 100644 index 000000000000..529c163f3a7e --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PortWriter.java @@ -0,0 +1,107 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.StringUtils; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import io.druid.guice.annotations.Self; +import io.druid.server.DruidNode; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.StandardOpenOption; + + +public class PortWriter +{ + private static final Logger log = new Logger(PortWriter.class); + private final DruidNode node; + private volatile File portFile; + private volatile boolean started = false; + + @Inject + public PortWriter(@Self DruidNode node) + { + this.node = node; + } + + @LifecycleStart + public synchronized void start() + { + if (started) { + throw new ISE("Already started"); + } + int port = node.getPort(); + final File portFile = new File(TierLocalTaskRunner.PORT_FILE_NAME); + log.info("Writing port [%d] to [%s]", node.getPort(), portFile.getAbsoluteFile()); + if (portFile.exists()) { + throw new ISE("Port file [%s] already exists! cannot start already started", portFile); + } + final String portString = Integer.toString(port); + log.debug("Writing port [%d] to [%s]", port, portFile); + final ByteBuffer buffer = ByteBuffer.wrap(StringUtils.toUtf8(portString)); + try (FileChannel portFileChannel = FileChannel.open( + portFile.toPath(), + StandardOpenOption.CREATE_NEW, + StandardOpenOption.READ, + StandardOpenOption.WRITE + )) { + final FileLock fileLock = portFileChannel.lock(); + try { + while (buffer.hasRemaining()) { + portFileChannel.write(buffer); + } + } + finally { + fileLock.release(); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + this.portFile = portFile; + started = true; + } + + @LifecycleStop + public synchronized void stop() + { + if (!started) { + log.info("Already stopped, ignoring stop request"); + } + log.info("Stopping"); + final File portFile = this.portFile; + if (portFile != null) { + log.debug("Erasing port"); + if (portFile.exists() && !portFile.delete()) { + log.warn("Could not delete port file [%s]", portFile); + } + } + started = false; + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PostingTierRemoteTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PostingTierRemoteTaskRunner.java new file mode 100644 index 000000000000..1007aba8534f --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PostingTierRemoteTaskRunner.java @@ -0,0 +1,59 @@ +package io.druid.indexing.overlord; + +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import com.google.inject.Inject; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import io.druid.guice.annotations.Global; +import io.druid.indexing.common.task.Task; +import io.druid.server.DruidNode; + +import javax.annotation.Nullable; + +public class PostingTierRemoteTaskRunner extends AbstractTierRemoteTaskRunner +{ + private static final Logger LOG = new Logger(PostingTierRemoteTaskRunner.class); + + @Inject + public PostingTierRemoteTaskRunner( + TierTaskDiscovery tierTaskDiscovery, + @Global HttpClient httpClient, + TaskStorage taskStorage, + ScheduledExecutorFactory executorFactory + ) + { + super(tierTaskDiscovery, httpClient, taskStorage, executorFactory); + } + + @Override + protected void launch(SettableFuture future, Task task) + { + throw new UnsupportedOperationException("TODO"); + } + + @Override + protected ListenableFuture kill(final DruidNode node) + { + final ListenableFuture softKillFuture = super.kill(node); + final SettableFuture hardKillFuture = SettableFuture.create(); + Futures.addCallback(softKillFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + LOG.debug("Shutdown request on [%s] succeeded", node); + } + + @Override + public void onFailure(Throwable t) + { + LOG.warn(t, "Shutdown request on [%s] failed, attempting hard-kill"); + } + }); + return hardKillFuture; + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java new file mode 100644 index 000000000000..d122eb548915 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java @@ -0,0 +1,1539 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Throwables; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.io.ByteSource; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.Pair; +import com.metamx.common.RE; +import com.metamx.common.StringUtils; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.emitter.EmittingLogger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.response.ClientResponse; +import com.metamx.http.client.response.HttpResponseHandler; +import com.sun.tools.attach.AttachNotSupportedException; +import com.sun.tools.attach.VirtualMachine; +import com.sun.tools.attach.VirtualMachineDescriptor; +import io.druid.concurrent.Execs; +import io.druid.curator.discovery.ServiceAnnouncer; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Self; +import io.druid.indexing.common.TaskLocation; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.common.tasklogs.LogUtils; +import io.druid.indexing.overlord.autoscaling.ScalingStats; +import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; +import io.druid.indexing.overlord.resources.DeadhandResource; +import io.druid.indexing.worker.config.WorkerConfig; +import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogPusher; +import io.druid.tasklogs.TaskLogStreamer; +import org.apache.commons.io.FileUtils; +import org.jboss.netty.handler.codec.http.HttpChunk; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.joda.time.Duration; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Properties; +import java.util.Scanner; +import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +/** + * The TierLocalTaskRunner is a gateway between general cluster task information and forked stuff locally on a + * single machine. It communicate existence via PORT_FILE_NAME in the task attempt directory. + * Using the PORT_FILE_NAME the TierLocalTaskRunner can identify peons on the local machine. + * The TierLocalTaskRunner is able to restart without affecting the state of the peons. + * The Peons run some special magic in io.druid.indexing.overlord.TierModule to expose an ability to shutdown the VMs + * cleanly. + */ +public class TierLocalTaskRunner implements TaskRunner, TaskLogStreamer +{ + private enum ForkingTaskRunnerState + { + INITIALIZING, + STARTING, + STARTED, + STOPPING, + STOPPED + } + + public static final String TASKID_PROPERTY = "io.druid.indexing.worker.taskid"; + public static final String PORT_FILE_NAME = "task.port"; + public static final String LOG_FILE_NAME = "task.log"; + public static final String TASK_FILE_NAME = "task.json"; + public static final String STATUS_FILE_NAME = "status.json"; + public static final String DEADHAND_FILE_NAME = "delete_me_to_terminate"; + private static final EmittingLogger log = new EmittingLogger(TierLocalTaskRunner.class); + protected static final String CHILD_PROPERTY_PREFIX = "druid.indexer.fork.property."; + private static final int MAX_DELETE_RETRIES = 3; // How many times should we try to delete the attempt dir on cleanup + private static final String LONGEST_POSITIVE_INTEGER = String.format("%d", Integer.MAX_VALUE); + + private final TierLocalTaskRunnerConfig config; + private final TaskConfig taskConfig; + private final Properties props; + private final TaskLogPusher taskLogPusher; + private final DruidNode node; + private final ObjectMapper jsonMapper; + private final PortFinder portFinder; + private final HttpClient httpClient; + private final ServiceAnnouncer serviceAnnouncer; + private final ConcurrentMap tasks = new ConcurrentHashMap<>(); + private final AtomicReference state = new AtomicReference<>(ForkingTaskRunnerState.INITIALIZING); + protected final ListeningExecutorService exec; // protected for unit tests + protected final ScheduledExecutorService heartbeatExec; + protected final AtomicReference> heartbeatFuture = new AtomicReference<>(null); + private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>(); + private final Object startStopLock = new Object(); + + @Inject + public TierLocalTaskRunner( + final TierLocalTaskRunnerConfig config, + final TaskConfig taskConfig, + final WorkerConfig workerConfig, + final Properties props, + final TaskLogPusher taskLogPusher, + final ObjectMapper jsonMapper, + final @Self DruidNode node, + final @Global HttpClient httpClient, + final ServiceAnnouncer serviceAnnouncer + ) + { + this.config = config; + this.taskConfig = taskConfig; + this.props = props; + this.taskLogPusher = taskLogPusher; + this.jsonMapper = jsonMapper; + this.node = node; + this.portFinder = new PortFinder(config.getStartPort()); + this.httpClient = httpClient; + this.serviceAnnouncer = serviceAnnouncer; + this.exec = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + workerConfig.getCapacity(), + "ForkingTaskWatcherExecutor-%d" + ) + ); + this.heartbeatExec = Executors.newScheduledThreadPool( + workerConfig.getCapacity() + 1, // +1 for the overall heartbeat submitter + Execs.makeThreadFactory("localHeartbeatEmitter-%s") + ); + if (config.getHeartbeatLocalNetworkTimeout() < config.getDelayBetweenHeartbeatBatches() * 3) { + log.warn( + "Heartbeats to peon may overlap. heartbeatLocalNetworkTimeout [%d] too high, or maxHeartbeatRetries [%d] too high, or heartbeatTimeLimit [%d] too low", + config.getHeartbeatLocalNetworkTimeout(), + config.getMaxHeartbeatRetries(), + config.getHeartbeatTimeLimit() + ); + } + } + + /** + * "Attach" to a given task ID. This returns a future which will wait for the given task to end and return the result. + * This method assumes a task is already running, and attempts to wait for it to finish. + * This will also create an asynchronous task to do the cleanup and reporting of the task. + * + * @param taskId The task ID of interest + * @param leaderLatch A leader latch to wait on to determine if this attach instance is the winner in races. + * @param exec The executor service to submit tasks to + * + * @return A ListenableFuture which will wait on taskId to complete then return its status + */ + private ListenableFuture attach( + final String taskId, + final CountDownLatch leaderLatch, + ListeningExecutorService exec + ) + { + final AtomicReference workItemAtomicReference = new AtomicReference<>(null); + final ListenableFuture future = exec.submit( + new Callable() + { + @Override + public TaskStatus call() throws Exception + { + if (leaderLatch != null) { + leaderLatch.await(); + } + final ForkingTaskRunnerWorkItem workItem = tasks.get(taskId); + if (workItem == null) { + throw new NullPointerException(String.format("Task [%s] not found", taskId)); + } + workItemAtomicReference.set(workItem); + final ProcessHolder processHolder = workItem.processHolder.get(); + if (processHolder == null) { + throw new NullPointerException(String.format("Task [%s] has no process holder, cannot attach!", taskId)); + } + + processHolder.awaitShutdown(Long.MAX_VALUE); + + // Give status file a little time to update before we call it failed + final File statusFile = new File( + getTaskAttemptDir(processHolder.taskId, processHolder.attemptId), + STATUS_FILE_NAME + ); + // Handle race condition on task exiting and updating file. Wait for modifications in the output directory + if (!statusFile.exists() || statusFile.length() == 0) { + final Path statusPathParent = statusFile.toPath().getParent(); + long startTime = System.currentTimeMillis(); + try (WatchService watchService = statusPathParent.getFileSystem().newWatchService()) { + statusPathParent.register( + watchService, + StandardWatchEventKinds.ENTRY_CREATE, + StandardWatchEventKinds.ENTRY_MODIFY + ); + while (!statusFile.exists() || statusFile.length() == 0) { + final WatchKey key = watchService.poll(1, TimeUnit.SECONDS); + if (key != null) { + log.debug("Watch events [%s] for task [%s]", key, taskId); + } + if (System.currentTimeMillis() - startTime > config.getSoftShutdownTimeLimit()) { + break; // error case will be handled below + } + } + } + } + + if (statusFile.exists() && statusFile.length() > 0) { + final TaskStatus status = jsonMapper.readValue(statusFile, TaskStatus.class); + log.info("Task [%s] exited with status [%s]", processHolder.taskId, status); + return TaskStatus.fromCode(processHolder.taskId, status.getStatusCode()); + } else { + log.warn("Unable to find status file at [%s]. Reporting as failed", statusFile); + return TaskStatus.failure(processHolder.taskId); + } + } + } + ); + Futures.addCallback( + future, + new FutureCallback() + { + @Override + public void onSuccess(TaskStatus result) + { + // Success of retrieving task status, not success of task + final ForkingTaskRunnerWorkItem workItem = workItemAtomicReference.get(); + final ProcessHolder processHolder = workItem.processHolder.get(); + uploadLogAndCleanDir(taskId, processHolder.attemptId); + portFinder.markPortUnused(processHolder.port); + if (!tasks.remove(taskId, workItem)) { + log.error("Task state corrupted, work items did not match for [%s] when cleaning up", taskId); + } + } + + @Override + public void onFailure(Throwable t) + { + final ForkingTaskRunnerWorkItem workItem = workItemAtomicReference.get(); + if (workItem == null) { + if (t instanceof CancellationException) { + log.debug("Task [%s] did not have work item set. Probably didn't win leader election", taskId); + } else { + log.error(t, "Error in attaching to task [%s]", taskId); + } + return; + } + final ProcessHolder processHolder = workItem.processHolder.get(); + if (processHolder == null) { + log.error("Task [%s] has no process holder, cannot attach!", taskId); + return; + } + try { + portFinder.markPortUnused(processHolder.port); + if (t instanceof InterruptedException) { + log.info("Task watcher for [%s] was interrupted", processHolder); + } else { + log.error(t, "Task watcher for [%s] had an error on attaching", processHolder); + } + } + finally { + if (!tasks.remove(processHolder.taskId, workItem)) { + log.warn("work item didn't match entry in tasks for [%s]", processHolder.taskId); + } + } + } + }, + exec + ); + return future; + } + + private void uploadLogAndCleanDir(String taskId, String attemptId) + { + final File taskAttemptDir = getTaskAttemptDir(taskId, attemptId); + final File taskDir = getTaskDir(taskId); + final File logFile = getLogFile(taskId, attemptId); + try { + taskLogPusher.pushTaskLog(taskId, logFile); + int remainingTries = MAX_DELETE_RETRIES; + while (taskAttemptDir.exists() && remainingTries-- > 0) { + try { + FileUtils.deleteDirectory(taskAttemptDir); + log.debug("Cleaned up [%s]", taskAttemptDir); + } + // IOException on race condition on deleting dir, IAE if dir is eliminated between exists check and deleteDirectory's exists check + catch (IOException | IllegalArgumentException ex) { + log.debug(ex, "Error cleaning up files at [%s]", taskAttemptDir); + } + } + if (taskAttemptDir.exists()) { + log.error("Could not cleanup directory [%s]", taskAttemptDir); + } + final File lockFile = new File(taskDir, "lock"); + if (lockFile.exists() && !lockFile.delete()) { + if (lockFile.exists()) { + log.warn("Could not clean out lock file in [%s]", taskDir); + } + } + if (!taskDir.delete()) { + log.debug("Could not clear task directory [%s]", taskDir); + } + } + catch (IOException ex) { + log.error(ex, "Error pushing log file [%s]", logFile); + } + } + + @Override + public List>> restore() + { + // No special case here, any old tasks are already added to the known tasks at startup + return ImmutableList.of(); + } + + @Override + public void registerListener(TaskRunnerListener listener, Executor executor) + { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public void unregisterListener(String listenerId) + { + throw new UnsupportedOperationException("Not yet supported"); + } + + // General workflow: + // 1. Create a future which waits for leader election before proceeding + // 2. Check for leadership by ConcurrentMap putIfAbsent + // 3. If leader, allow future to continue, else cancel future (which is waiting on latch) + // 4. Future creates process + // 5. Future runs process + // 6. Future calls attach() task in order to wait for task completion. + @Override + public ListenableFuture run(final Task task) + { + final CountDownLatch leaderLatch = new CountDownLatch(1); + + // Submit a new task which will launch the job, then wait on an attach to the job + final ListenableFuture startingFuture = exec.submit( + new Callable() + { + @Override + public TaskStatus call() + { + try { + leaderLatch.await(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + final int childPort = portFinder.findUnusedPort(); + final File attemptDir = getNewTaskAttemptDir(task.getId()); + final String attemptUUID = attemptDir.getName(); + final ProcessHolder processHolder = new ProcessHolder(task.getId(), attemptUUID, node.getHost(), childPort); + final Path attemptPath = attemptDir.toPath(); + + log.debug("Created directory [%s] for task [%s]", attemptDir, task.getId()); + + try { + final File taskFile = new File(attemptDir, TASK_FILE_NAME); + final File statusFile = new File(attemptDir, STATUS_FILE_NAME); + final File logFile = new File(attemptDir, LOG_FILE_NAME); + final File deadhandFile = new File(attemptDir, DEADHAND_FILE_NAME); + + if (!taskFile.exists() && !taskFile.createNewFile()) { + throw new IOException(String.format("Could not create file [%s]", taskFile)); + } + if (!statusFile.exists() && !statusFile.createNewFile()) { + throw new IOException(String.format("Could not create file [%s]", statusFile)); + } + if (!logFile.exists() && !logFile.createNewFile()) { + throw new IOException(String.format("Could not create file [%s]", logFile)); + } + if (!deadhandFile.exists() && !deadhandFile.createNewFile()) { + throw new IOException(String.format("Could not create file [%s]", deadhandFile)); + } + + // time to adjust process holders + final ForkingTaskRunnerWorkItem taskWorkItem = tasks.get(task.getId()); + + if (taskWorkItem == null) { + log.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); + throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); + } + + if (taskWorkItem.shutdown.get()) { + throw new IllegalStateException("Task has been shut down!"); + } + + if (taskWorkItem.processHolder.get() != null) { + // Fail early, there is also a last second check later on + log.makeAlert("WTF?! TaskInfo already has a processHolder") + .addData("task", task.getId()) + .emit(); + throw new ISE("TaskInfo already has processHolder for task[%s]!", task.getId()); + } + + final List command = Lists.newArrayList(); + final String childHost = node.getHost(); + final String taskClasspath; + if (task.getClasspathPrefix() != null && !task.getClasspathPrefix().isEmpty()) { + taskClasspath = Joiner.on(File.pathSeparator).join( + task.getClasspathPrefix(), + config.getClasspath() + ); + } else { + taskClasspath = config.getClasspath(); + } + + command.add(config.getJavaCommand()); + command.add("-cp"); + command.add(taskClasspath); + + Iterables.addAll( + command, + new QuotableWhiteSpaceSplitter(config.getJavaOpts()) + ); + Iterables.addAll( + command, + config.getJavaOptsArray() + ); + + for (String propName : props.stringPropertyNames()) { + for (String allowedPrefix : config.getAllowedPrefixes()) { + if (propName.startsWith(allowedPrefix)) { + command.add( + String.format( + "-D%s=%s", + propName, + props.getProperty(propName) + ) + ); + } + } + } + + // Override child JVM specific properties + for (String propName : props.stringPropertyNames()) { + if (propName.startsWith(CHILD_PROPERTY_PREFIX)) { + command.add( + String.format( + "-D%s=%s", + propName.substring(CHILD_PROPERTY_PREFIX.length()), + props.getProperty(propName) + ) + ); + } + } + + command.add(String.format("-Ddruid.host=%s", childHost)); + command.add(String.format("-Ddruid.port=%d", childPort)); + + command.add(String.format("-D" + TASKID_PROPERTY + "=%s", task.getId())); + + command.add("io.druid.cli.Main"); + command.add("tier"); + command.add("fork"); + + String nodeType = task.getNodeType(); + if (nodeType != null) { + command.add("--nodeType"); + command.add(nodeType); + } + + // Needed for legacy CliPeon support + command.add(taskFile.getAbsolutePath()); + command.add(statusFile.getAbsolutePath()); + + jsonMapper.writeValue(taskFile, task); + try (WatchService watchService = attemptPath.getFileSystem().newWatchService()) { + attemptPath.register( + watchService, + StandardWatchEventKinds.ENTRY_CREATE, + StandardWatchEventKinds.ENTRY_MODIFY + ); + log.info("Running command: %s", Joiner.on(" ").join(command)); + // Process can continue running in the background. We monitor via files rather than Process + final Process process = new ProcessBuilder(command) + .redirectError(logFile) + .redirectOutput(logFile) + .directory(attemptDir) + .start(); + if (!taskWorkItem.processHolder.compareAndSet(null, processHolder)) { + final String msg = String.format( + "WTF!? Expected empty process holder and found [%s]", + taskWorkItem.processHolder.get() + ); + log.makeAlert("%s", msg).emit(); + throw new ISE("%s", msg); + } + + log.info("Logging task %s output to: %s", task.getId(), logFile); + // Wait for files to be modified by task starting + log.debug("Waiting for task [%s] to start", processHolder); + watchService.take();// Should only be modified by task + log.debug("Waiting for task [%s] to finish", processHolder); + + return attach(task.getId(), leaderLatch, MoreExecutors.sameThreadExecutor()).get(); + } + } + catch (InterruptedException e) { + log.info("Interrupted while waiting for task [%s]", processHolder); + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + final Throwable eCause = e.getCause(); + if (eCause instanceof InterruptedException) { + log.info(e, "Attaching interrupted for [%s]", processHolder); + Thread.currentThread().interrupt(); + } else { + log.info(e, "Exception during execution of attach for [%s]", processHolder); + } + throw Throwables.propagate(e); + } + catch (Throwable t) { + log.info(t, "Exception caught during forking of [%s]", processHolder); + throw Throwables.propagate(t); + } + } + } + ); + try { + final ForkingTaskRunnerWorkItem workItem = new ForkingTaskRunnerWorkItem(task.getId(), startingFuture); + // Leader election for task id + final ForkingTaskRunnerWorkItem leaderItem = tasks.putIfAbsent(task.getId(), workItem); + if (leaderItem != null) { + if (!startingFuture.cancel(true)) { + log.makeAlert("Task [%s] had a race condition and couldn't stop!", task.getId()).emit(); + } + log.warn("Already have task id [%s], returning prior task instead", task.getId()); + return leaderItem.getResult(); + } else { + return workItem.getResult(); + } + } + finally { + leaderLatch.countDown(); + } + } + + + // This assumes that no task directories can be created except in the ForkingTaskRunner, + // And that the ForkingTaskRunner has exclusive ownership of the directory structure + @Override + @LifecycleStart + public void start() + { + synchronized (startStopLock) { + if (!state.compareAndSet(ForkingTaskRunnerState.INITIALIZING, ForkingTaskRunnerState.STARTING)) { + throw new ISE( + "Invalid state for starting: Expected [%s] found [%s]", + ForkingTaskRunnerState.INITIALIZING, + state.get() + ); + } + + populateMissingTasksFromDir(); + serviceAnnouncer.announce(node); + final Future heartbeatFuture = heartbeatExec.scheduleAtFixedRate( + new Runnable() + { + @Override + public void run() + { + for (final ForkingTaskRunnerWorkItem workItem : tasks.values()) { + if (workItem.shutdown.get()) { + continue; + } + final ProcessHolder holder = workItem.processHolder.get(); + if (holder == null) { + continue; + } + final int port = holder.port; + if (port == 0) { + continue; + } + + heartbeatExec.submit( + new Runnable() + { + @Override + public void run() + { + try { + final URL url = new URL( + "http", + "localhost", + port, + DeadhandResource.DEADHAND_PATH + ); + httpClient.go( + new Request(HttpMethod.POST, url), + new HttpResponseHandler() + { + @Override + public ClientResponse handleResponse(HttpResponse response) + { + log.debug("heartbeat response for port [%d]: [%s]", port, response.getStatus()); + return ClientResponse.finished(null); + } + + @Override + public ClientResponse handleChunk( + ClientResponse clientResponse, HttpChunk chunk + ) + { + log.trace("chunk response for heartbeat on port [%d]", port); + return clientResponse; + } + + @Override + public ClientResponse done(ClientResponse clientResponse) + { + log.trace("done with heartbeat for port [%d]", port); + return clientResponse; + } + + @Override + public void exceptionCaught(ClientResponse clientResponse, Throwable e) + { + log.error(e, "Error in url [%s]", url); + } + }, + Duration.millis(config.getHeartbeatLocalNetworkTimeout()) + ).get(); // So we don't clog the pipeline + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (Exception e) { + log.warn(e, "Error in submitting heartbeat on port [%d]", port); + } + } + } + ); // We don't care about failures + } + } + }, + 0, + config.getDelayBetweenHeartbeatBatches(), + TimeUnit.MILLISECONDS + ); + if (!this.heartbeatFuture.compareAndSet(null, heartbeatFuture)) { + if (!heartbeatFuture.cancel(true)) { + log.makeAlert("Error canceling duplicate heartbeat emitter").emit(); + } + throw new ISE("Heartbeat future was not null"); + } + if (!state.compareAndSet(ForkingTaskRunnerState.STARTING, ForkingTaskRunnerState.STARTED)) { + throw new ISE( + "Someone is screwing around with my start state! Expected [%s] found [%s]", + ForkingTaskRunnerState.STARTING, + state.get() + ); + } + log.info("Started"); + } + } + + @Override + @LifecycleStop + public void stop() + { + synchronized (startStopLock) { + final ForkingTaskRunnerState s = state.get(); + if (ForkingTaskRunnerState.STOPPED.equals(s) || ForkingTaskRunnerState.STOPPING.equals(s)) { + log.debug("Already stopped, ignoring"); + return; + } + if (!state.compareAndSet(ForkingTaskRunnerState.STARTED, ForkingTaskRunnerState.STOPPING)) { + if (ForkingTaskRunnerState.STOPPING.equals(s) || ForkingTaskRunnerState.STOPPED.equals(s)) { + log.info("Stop called multiple times. Ignoring stop request"); + return; + } else { + throw new ISE("Invalid state to stop. Expected [%s] found [%s]", ForkingTaskRunnerState.STARTED, s); + } + } + serviceAnnouncer.unannounce(node); + exec.shutdown(); + final Future heartbeatFuture = this.heartbeatFuture.get(); + if (heartbeatFuture != null) { + heartbeatFuture.cancel(false); + } + if (!(this.heartbeatFuture.compareAndSet(heartbeatFuture, null))) { + log.error("Illegal state, heartbeat changed during stop()"); + } + heartbeatExec.shutdown(); + if (!state.compareAndSet(ForkingTaskRunnerState.STOPPING, ForkingTaskRunnerState.STOPPED)) { + throw new ISE( + "Someone is screwing with my shutdown state! Expected [%s] found [%s]", + ForkingTaskRunnerState.STOPPING, + state.get() + ); + } + log.info("Stopped"); + } + } + + @Override + public void shutdown(final String taskid) + { + final ForkingTaskRunnerWorkItem taskInfo = tasks.get(taskid); + + if (taskInfo == null) { + log.info("Ignoring request to cancel unknown task: %s", taskid); + return; + } + + if (!taskInfo.shutdown.compareAndSet(false, true)) { + log.warn("Someone already shut down task [%s]. Ignoring request", taskid); + return; + } + + final ProcessHolder processHolder = taskInfo.processHolder.get(); + if (processHolder == null) { + log.wtf("Task has no process holder!?"); + return; + } + // Check to see if foreign process needs to be killed + if (processHolder.deadhandFile.exists()) { + if (!processHolder.deadhandFile.delete() && processHolder.deadhandFile.exists()) { + throw new RE("Could not remove file at [%s]", processHolder.deadhandFile); + } + log.info("Attempting shutdown via deletion of [%s]", processHolder.deadhandFile); + } + try { + try { + processHolder.awaitShutdown(config.getSoftShutdownTimeLimit()); + } + catch (TimeoutException e) { + log.info( + "Timed out waiting for clean shutdown on task [%s]. Forcing shutdown...", + taskInfo.processHolder.get() + ); + if (!forceKill(processHolder)) { + if (processHolder.taskPortFile.exists()) { + throw new RuntimeException("Unable to shutdown task!"); + } else { + log.info("Task shutdown on its own"); + } + } + } + taskInfo.getResult().get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + if (e.getCause() instanceof InterruptedException) { + log.info("Interrupted while waiting for shutdown"); + return; + } + throw Throwables.propagate(e); + } + } + + /** + * Unix (and maybe Oracle VM) specific killer for processes + * + * @param processHolder The process holder of interest + * + * @return True if the task was killed via this method, false otherwise. + * + * @throws InterruptedException If the waiting on system `kill` commands is interrupted. + */ + private boolean forceKill(final ProcessHolder processHolder) throws InterruptedException + { + final String portString = Integer.toString(processHolder.port); + final List vms = ImmutableList.copyOf( + Collections2.filter( + VirtualMachine.list(), + new Predicate() + { + @Override + public boolean apply(VirtualMachineDescriptor input) + { + try { + return portString.equals( + input.provider() + .attachVirtualMachine(input) + .getSystemProperties() + .getProperty("druid.port") + ); + } + catch (IOException | AttachNotSupportedException e) { + log.warn(e, "Could not read property from vm"); + return false; + } + } + } + ) + ); + if (vms.isEmpty()) { + log.warn("Could not find vm for taskid [%s] using port [%d]!", processHolder.taskId, processHolder.port); + return false; + } + final VirtualMachineDescriptor vmd = vms.get(0); + try { + final int pid = Integer.parseInt(vmd.id()); + log.info("Forcing kill of task [%s] on pid [%d]", processHolder.taskId, pid); + } + catch (NumberFormatException e) { + log.error("Could not find pid for task [%s]. VM id [%s] is not an integer", processHolder.taskId, vmd.id()); + return false; + } + final File tmpFile; + try { + tmpFile = File.createTempFile("kill_output", ".tmp"); + } + catch (IOException e) { + log.error(e, "Could not create output file to kill task [%s] on port [%d]", processHolder.taskId, vmd.id()); + return false; + } + try { + Process killingProcess = new ProcessBuilder(ImmutableList.of("kill", "-15", vmd.id())) + .redirectOutput(tmpFile) + .redirectError(tmpFile) + .start(); + int retval = killingProcess.waitFor(); + if (retval == 0) { + processHolder.awaitShutdown(config.getSoftShutdownTimeLimit()); + return true; + } + try (InputStream inputStream = new FileInputStream(tmpFile)) { + Scanner scanner = new Scanner(inputStream).useDelimiter("\\A"); + log.error( + "Term of pid [%s] did not succeed with code [%d]: [%s]", + vmd.id(), + retval, + scanner.hasNext() ? scanner.next() : "null" + ); + } + tmpFile.delete(); + tmpFile.createNewFile(); + killingProcess = new ProcessBuilder(ImmutableList.of("kill", "-9", vmd.id())) + .redirectOutput(tmpFile) + .redirectError(tmpFile) + .start(); + retval = killingProcess.waitFor(); + if (retval == 0) { + processHolder.awaitShutdown(config.getSoftShutdownTimeLimit()); + return true; + } + try (InputStream inputStream = new FileInputStream(tmpFile)) { + Scanner scanner = new Scanner(inputStream).useDelimiter("\\A"); + log.error( + "Kill of pid [%s] did not succeed with code [%d]: [%s]", + vmd.id(), + retval, + scanner.hasNext() ? scanner.next() : "null" + ); + } + return false; + } + catch (IOException | TimeoutException e) { + throw Throwables.propagate(e); + } + finally { + tmpFile.delete(); + } + } + + @Override + public Collection getRunningTasks() + { + return Collections2.transform( + Collections2.filter( + tasks.values(), + new Predicate() + { + @Override + public boolean apply(ForkingTaskRunnerWorkItem input) + { + return input.processHolder.get() != null; + } + } + ), + new Function() + { + @Override + public TaskRunnerWorkItem apply(ForkingTaskRunnerWorkItem input) + { + return input; + } + } + ); + } + + @Override + public Collection getPendingTasks() + { + return Collections2.transform( + Collections2.filter( + tasks.values(), + new Predicate() + { + @Override + public boolean apply(ForkingTaskRunnerWorkItem input) + { + return input.processHolder.get() == null; + } + } + ), + new Function() + { + @Override + public TaskRunnerWorkItem apply(ForkingTaskRunnerWorkItem input) + { + return input; + } + } + ); + } + + @Override + public Collection getKnownTasks() + { + return ImmutableList.copyOf(tasks.values()); + } + + @Override + public Optional getScalingStats() + { + return Optional.absent(); + } + + @Override + public Optional streamTaskLog(final String taskid, final long offset) + { + final ForkingTaskRunnerWorkItem taskWorkItem = tasks.get(taskid); + + if (taskWorkItem == null) { + return Optional.absent(); + } + + final ProcessHolder processHolder = taskWorkItem.processHolder.get(); + + if (processHolder == null) { + return Optional.absent(); + } + + final File logFile = getLogFile(processHolder); + + if (!logFile.exists()) { + return Optional.absent(); + } + + return Optional.of( + new ByteSource() + { + @Override + public InputStream openStream() throws IOException + { + return LogUtils.streamFile(logFile, offset); + } + } + ); + } + + private Collection getAttemptDirs() + { + final File baseDir = taskConfig.getBaseTaskDir(); + final File[] taskDirFileArray = baseDir.listFiles(); + final Collection taskDirFileList = + taskDirFileArray == null ? + ImmutableList.of() : + Collections2.filter( + Arrays.asList( + taskDirFileArray + ), new Predicate() + { + @Override + public boolean apply(File input) + { + return input.exists() && input.isDirectory(); + } + } + ); + if (taskDirFileList.isEmpty()) { + log.info("No task dirs found in [%s]", baseDir); + return ImmutableList.of(); + } + return taskDirFileList; + } + + private void populateMissingTasksFromDir() + { + if (!ForkingTaskRunnerState.STARTING.equals(state.get())) { + // This might be safe to do, but this method assumes this is not the case + throw new ISE("Cannot populate tasks from dirs once ForkingTaskRunner has been started"); + } + + final Collection taskDirFileList = getAttemptDirs(); + if (taskDirFileList.isEmpty()) { + log.info("No prior task attempts found"); + return; + } + log.debug("Looking for files in %s", taskDirFileList); + final ListeningExecutorService lookingExecutor = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + taskDirFileList.size(), + "localTaskDiscovery-%s" + ) + ); + final ArrayList> futures = Lists.newArrayListWithExpectedSize(taskDirFileList.size()); + // For the task directories, look for attempt directories + for (final File potentialTaskDir : taskDirFileList) { + futures.add( + lookingExecutor.submit( + new Runnable() + { + @Override + public void run() + { + final File[] taskAttemptDirFileArray = potentialTaskDir.listFiles(); + final Collection taskAttemptDirFileList = // Only directories containing a non-zero TASK_FILE_NAME file + taskAttemptDirFileArray == null ? + ImmutableList.of() : + Collections2.filter( + Arrays.asList(taskAttemptDirFileArray), + new Predicate() + { + @Override + public boolean apply(File input) + { + return input.isDirectory() && input.listFiles( + new FilenameFilter() + { + @Override + public boolean accept(File dir, String name) + { + return TASK_FILE_NAME.equals(name) + && new File(dir, TASK_FILE_NAME).length() + > 0; + } + } + ) != null; // Did we find any? + } + } + ); + if (taskAttemptDirFileList.isEmpty()) { + log.info( + "Directory [%s] has no viable task attempts, attempting to cleanup if empty", + potentialTaskDir + ); + if (!potentialTaskDir.delete()) { + log.warn("Could not clean up [%s]", potentialTaskDir); + } + return; + } else { + log.debug("Found viable task in [%s]", potentialTaskDir); + } + // Find latest attempt in directory + long last_attempt = 0; + File latestAttemptDir = null; + for (File taskAttemptDir : taskAttemptDirFileList) { + if (!taskAttemptDir.isDirectory()) { + log.debug("Skipping non-directory [%s]", taskAttemptDir); + continue; + } + try { + final long check_attempt = Long.parseLong(taskAttemptDir.getName()); + if (check_attempt > last_attempt) { + latestAttemptDir = getTaskAttemptDir(taskAttemptDir.getName(), check_attempt); + last_attempt = check_attempt; + } + if (latestAttemptDir == null) { + latestAttemptDir = taskAttemptDir; + } else { + if (latestAttemptDir.lastModified() < taskAttemptDir.lastModified()) { + latestAttemptDir = taskAttemptDir; + } + } + } + catch (NumberFormatException e) { + log.info(e, "Skipping unparsable directory [%s]", taskAttemptDir); + } + } + if (latestAttemptDir == null) { + log.warn("Didn't find any viable attempts among %s", taskAttemptDirFileList); + return; + } + + //------------------------------------------------------------------ Load up data from suspected good attempt dirs + + // We already checked earlier that this exists and is non zero + final File taskFile = new File(latestAttemptDir, TASK_FILE_NAME); + final Task task; + try { + task = jsonMapper.readValue(taskFile, Task.class); + } + catch (IOException e) { + log.makeAlert(e, "Corrupted task file at [%s]", taskFile).emit(); + return; + } + + final File statusFile = new File(latestAttemptDir, STATUS_FILE_NAME); + + if (!statusFile.exists()) { + // Shouldn't be missing unless there's corruption somehow. + log.makeAlert("Status file [%s] is missing ", statusFile).emit(); + return; + } + + final File portFile = new File(latestAttemptDir, PORT_FILE_NAME); + Integer port = null; + try { + port = getPort(portFile, task.getId()); + } + catch (IOException e) { + log.makeAlert( + e, + "Error reading port file [%s] for task [%s] in dir [%s]", + portFile, + task.getId(), + latestAttemptDir + ).emit(); + return; + } + if (port == null) { + // At this point there should be one of two scenarios: + // A) The peon is still starting up and hasn't written the port file yet + // B) The peon has exited between ForkingTaskRunner instances + // We'll handle A first + final Path portPathParent = portFile.toPath().getParent(); + try (WatchService watchService = portPathParent.getFileSystem().newWatchService()) { + final long start = System.currentTimeMillis(); + while (port == null) { + port = getPort(portFile, task.getId()); + watchService.poll(1, TimeUnit.SECONDS); + if (System.currentTimeMillis() - start > config.getSoftShutdownTimeLimit()) { + log.info( + "Timeout exceeded while waiting for task [%s] to publish its port information", + task.getId() + ); + break; + } + } + } + catch (IOException e) { + log.makeAlert( + e, + "Error reading port file [%s] for task [%s] in dir [%s]", + portFile, + task.getId(), + latestAttemptDir + ).emit(); + return; // Skip so it can be investigated. On IOException cleanup probably won't help anyways + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + } + + if (port == null) { + // We think the task really is dead + log.debug("Found no port file for task [%s]. Uploading log and cleaning", task.getId()); + final CountDownLatch doneLatch = new CountDownLatch(1); + try { + final ListenableFuture future; + final ForkingTaskRunnerWorkItem workItem; + try { + future = attach(task.getId(), doneLatch, exec); + workItem = new ForkingTaskRunnerWorkItem(task.getId(), future); + workItem.processHolder = new AtomicReference<>( + new ProcessHolder( + task.getId(), + latestAttemptDir.getName(), + node.getHost(), + 0 + ) + ); + tasks.put(task.getId(), workItem); + } + finally { + doneLatch.countDown(); + } + future.get(); + workItem.processHolder.get().awaitShutdown(100L); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (TimeoutException | ExecutionException e) { + log.makeAlert(e, "Could upload data for task [%s] which finished between runs", task.getId()) + .emit(); + } + return; + } + final ProcessHolder processHolder = new ProcessHolder( + task.getId(), + latestAttemptDir.getName(), + node.getHost(), + port + ); + final CountDownLatch leaderLatch = new CountDownLatch(1); + final ForkingTaskRunnerWorkItem workItem = new ForkingTaskRunnerWorkItem( + task.getId(), + attach(task.getId(), leaderLatch, exec) + ); + try { + workItem.processHolder.set(processHolder); + if (tasks.putIfAbsent(task.getId(), workItem) != null) { + log.warn("Task [%s] already exists!", task.getId()); + workItem.getResult().cancel(true); + } else { + log.info("Found task [%s] in progress", processHolder); + } + } + finally { + leaderLatch.countDown(); + } + } + } + ) + ); + } + try { + // Exceptions that are recoverable are handled within the Runnable + // Anything that makes it up this high really iss a problem + Futures.allAsList(futures).get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + finally { + lookingExecutor.shutdown(); + } + log.info("Finished parsing potential tasks in %s", taskDirFileList); + } + + /** + * Returns the port number of the given task in the particular attempt directory. + * The PORT_FILE_NAME is the intended indicator for the JVM instance itself. + * Only the PEON is in charge of writing or deleting this file. + * + * @param portFile The file where the port number should be located as a UTF-8 string + * @param taskId The taskId for this request (used for logging) + * + * @return The port in use by the attempt or null if it is empty/not found + * + * @throws IOException on error in the underlying file (either corrupt file at the FS level or has nonsense contents) + */ + private static Integer getPort(File portFile, String taskId) throws IOException + { + log.debug("Checking port file [%s] for taskId [%s] in [%s]", portFile, taskId, portFile.getParent()); + Integer port = null; + if (!portFile.exists()) { + return null; + } + try (FileChannel portFileChannel = FileChannel.open( + portFile.toPath(), + StandardOpenOption.READ, + StandardOpenOption.WRITE // Required for lock + )) { + final ByteBuffer buffer; + final FileLock fileLock = portFileChannel.lock(); // To make sure the peon is done writing before we try to read + try { + final long fileSize = portFileChannel.size(); + if (fileSize > LONGEST_POSITIVE_INTEGER.length()) { + // Probably should never happen + throw new IOException( + String.format( + "port file [%s] for task [%s] is HUGE %d bytes", + portFile, + taskId, + fileSize + ) + ); + } + buffer = ByteBuffer.allocate((int) fileSize); + for (int totalRead = 0, thisRead; totalRead < fileSize; totalRead += thisRead) { + thisRead = portFileChannel.read(buffer); + } + buffer.rewind(); + } + finally { + fileLock.release(); + } + final String portString = StringUtils.fromUtf8(buffer.array()); + port = Integer.parseInt(portString); + } + catch (FileNotFoundException e) { + log.info(e, "Task [%s] attempt [%s] has no port file", taskId, portFile.getParent()); + return null; + } + catch (IOException | NumberFormatException e) { + if (portFile.exists()) { + // Something went wrong during write of value from peon's side + log.makeAlert(e, "Port file [%s] for task [%s] is corrupt", portFile, taskId).emit(); + throw new IOException(String.format("Corrupt port file [%s]", portFile), e); + } else { + // Exited during read + log.info(e, "Task [%s] attempt [%s] exited during read", taskId, portFile.getParent()); + } + return null; + } + return port; + } + + private File getTaskDir(String taskId) + { + return new File(taskConfig.getBaseTaskDir(), taskId); + } + + private File getNewTaskAttemptDir(String taskId) + { + final File taskDir = getTaskDir(taskId); + if (!taskDir.exists()) { + if (!taskDir.mkdirs() && !taskDir.exists()) { + throw new RuntimeException(new IOException(String.format("Unable to create file at [%s]", taskDir))); + } + } + if (!taskDir.isDirectory()) { + throw new RuntimeException(new IOException(String.format("[%s] not a directory", taskDir))); + } + final File[] files = taskDir.listFiles(); + long attempt_num = 0; + if (files != null) { + for (File file : files) { + if (file.isDirectory()) { + try { + long attempt = Long.parseLong(file.getName()); + if (attempt > attempt_num) { + attempt_num = attempt; + } + } + catch (NumberFormatException e) { + log.debug(e, "couldn't parse directory [%s]", file); + } + } + } + } + File file; + long attempt = attempt_num + 1; + do { + file = getTaskAttemptDir(taskId, attempt++); + } while (!file.mkdirs()); + return file; + } + + private File getTaskAttemptDir(String taskId, long attempt_num) + { + Preconditions.checkArgument(attempt_num < 10_000, "attempt_num < 10_000"); + return new File(getTaskDir(taskId), String.format("%04d", attempt_num)); + } + + private File getTaskAttemptDir(String taskId, String attemptId) + { + return new File(getTaskDir(taskId), attemptId); + } + + private File getLogFile(String taskId, String attemptId) + { + return new File(getTaskAttemptDir(taskId, attemptId), LOG_FILE_NAME); + } + + private File getLogFile(ProcessHolder processHolder) + { + return getLogFile(processHolder.taskId, processHolder.attemptId); + } + + private static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem + { + private AtomicBoolean shutdown = new AtomicBoolean(false); + private AtomicReference processHolder = new AtomicReference<>(null); + + private ForkingTaskRunnerWorkItem( + String taskId, + ListenableFuture statusFuture + ) + { + super(taskId, statusFuture); + } + + @Override + public TaskLocation getLocation() + { + final ProcessHolder processHolder = this.processHolder.get(); + if (processHolder == null) { + return TaskLocation.unknown(); + } + return TaskLocation.create(processHolder.localhost, processHolder.port); + } + } + + private class ProcessHolder + { + private final String taskId; + private final String attemptId; + private final int port; + private final String localhost; + private final File attemptDir; + private final File taskPortFile; + private final File deadhandFile; + + private ProcessHolder(String taskId, String attemptId, String localhost, int port) + { + this.taskId = taskId; + this.attemptId = attemptId; + this.localhost = localhost; + this.port = port; + attemptDir = getTaskAttemptDir(taskId, attemptId); + taskPortFile = new File(attemptDir, PORT_FILE_NAME); + deadhandFile = new File(attemptDir, DEADHAND_FILE_NAME); + } + + public void awaitShutdown(long timeoutMS) throws InterruptedException, TimeoutException + { + final long startTime = System.currentTimeMillis(); + final Path taskPath = taskPortFile.toPath(); + boolean retry = true; + while (retry) { + try (WatchService watchService = taskPath.getFileSystem().newWatchService()) { + taskPath.getParent().register(watchService, StandardWatchEventKinds.ENTRY_DELETE); + while (taskPortFile.exists()) { + final long delta = System.currentTimeMillis() - startTime; + if (timeoutMS <= delta) { + throw new TimeoutException("Waiting for the right delete event"); + } + // Ignore result, we use a check for the file explicitly in the loop + watchService.poll(100, TimeUnit.MILLISECONDS); + } + retry = false; + } + catch (IOException e) { + log.warn(e, "Exception in watch service"); + if (System.currentTimeMillis() - startTime > timeoutMS) { + final TimeoutException timeoutException = new TimeoutException("Watch service error"); + timeoutException.addSuppressed(e); + throw timeoutException; + } + } + } + } + + @Override + public String toString() + { + return "ProcessHolder{" + + "taskId='" + taskId + '\'' + + ", attemptId='" + attemptId + '\'' + + ", port=" + port + + '}'; + } + } + + // If start() has finished + public boolean isStarted(boolean block) + { + final ForkingTaskRunnerState state; + if (block) { + synchronized (startStopLock) { + state = this.state.get(); + } + } else { + state = this.state.get(); + } + return ForkingTaskRunnerState.STARTED.equals(state); + } + + // If stop() has finished + public boolean isStopped() + { + return ForkingTaskRunnerState.STOPPED.equals(state.get()); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunnerFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunnerFactory.java new file mode 100644 index 000000000000..aba24f7cbc68 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunnerFactory.java @@ -0,0 +1,91 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import com.metamx.http.client.HttpClient; +import io.druid.curator.discovery.ServiceAnnouncer; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Self; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; +import io.druid.indexing.worker.config.WorkerConfig; +import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogPusher; + +import java.util.Properties; + +public class TierLocalTaskRunnerFactory implements TaskRunnerFactory +{ + final TierLocalTaskRunnerConfig config; + final TaskConfig taskConfig; + final WorkerConfig workerConfig; + final Properties props; + final TaskLogPusher taskLogPusher; + final ObjectMapper jsonMapper; + final + @Self + DruidNode node; + final + @Global + HttpClient httpClient; + final ServiceAnnouncer serviceAnnouncer; + + @Inject + public TierLocalTaskRunnerFactory( + final TierLocalTaskRunnerConfig config, + final TaskConfig taskConfig, + final WorkerConfig workerConfig, + final Properties props, + final TaskLogPusher taskLogPusher, + final ObjectMapper jsonMapper, + final @Self DruidNode node, + final @Global HttpClient httpClient, + final ServiceAnnouncer serviceAnnouncer + ) + { + this.config = config; + this.taskConfig = taskConfig; + this.workerConfig = workerConfig; + this.props = props; + this.taskLogPusher = taskLogPusher; + this.jsonMapper = jsonMapper; + this.node = node; + this.httpClient = httpClient; + this.serviceAnnouncer = serviceAnnouncer; + } + + @Override + public TaskRunner build() + { + return new TierLocalTaskRunner( + config, + taskConfig, + workerConfig, + props, + taskLogPusher, + jsonMapper, + node, + httpClient, + serviceAnnouncer + ); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java new file mode 100644 index 000000000000..a79ce4eaaeb8 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java @@ -0,0 +1,207 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.base.Optional; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.io.ByteSource; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; +import com.metamx.common.IAE; +import com.metamx.common.Pair; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.common.logger.Logger; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.autoscaling.ScalingStats; +import io.druid.indexing.overlord.autoscaling.TierRoutingManagementStrategy; +import io.druid.indexing.overlord.routing.TierRouteConfig; +import io.druid.tasklogs.TaskLogStreamer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executor; + +/** + * Be a proxy/mux to multiple task runners. + * Currently this is not very smart about task IDs, favoring a shotgun approach to things which require a taskID. + */ +public class TierRoutingTaskRunner implements TaskRunner, TaskLogStreamer +{ + private static final Logger LOG = new Logger(TierRoutingTaskRunner.class); + private final ConcurrentMap runnerMap = new ConcurrentHashMap<>(); + private final ListeningScheduledExecutorService scheduledExecutorService; + final TierRoutingManagementStrategy managementStrategy; + + @Inject + public TierRoutingTaskRunner( + Supplier configSupplier, + ScheduledExecutorFactory managementExecutorServiceFactory + ) + { + scheduledExecutorService = MoreExecutors.listeningDecorator(managementExecutorServiceFactory.create( + 1, + "TierRoutingManagement--%d" + )); + managementStrategy = new TierRoutingManagementStrategy(runnerMap, configSupplier, scheduledExecutorService); + } + + // For Unit Tests + protected ConcurrentMap getRunnerMap() + { + return runnerMap; + } + + @Override + public List>> restore() + { + // As per RemoteTaskRunner + return ImmutableList.of(); + } + + @Override + public void registerListener(TaskRunnerListener listener, Executor executor) + { + throw new UnsupportedOperationException("Not yet implemented"); + } + + @Override + public void unregisterListener(String listenerId) + { + throw new UnsupportedOperationException("Not yet implemented"); + } + + @Override + public ListenableFuture run(Task task) + { + final TaskRunner runner = managementStrategy.getRunner(task); + if (runner == null) { + throw new IAE("tier for task [%s] not found", task.getId()); + } + return managementStrategy.getRunner(task).run(task); + } + + @Override + public void shutdown(String taskid) + { + for (TaskRunner taskRunner : runnerMap.values()) { + try { + taskRunner.shutdown(taskid); + } + catch (Exception e) { + LOG.error(e, "Error shutting down task [%s]", taskid); + } + } + } + + @Override + public Collection getRunningTasks() + { + final Collection items = new ArrayList<>(); + for (TaskRunner runner : runnerMap.values()) { + try { + items.addAll(runner.getRunningTasks()); + } + catch (Exception e) { + LOG.error(e, "Error fetching running tasks"); + } + } + return items; + } + + @Override + public Collection getPendingTasks() + { + final Collection items = new ArrayList<>(); + for (TaskRunner runner : runnerMap.values()) { + try { + items.addAll(runner.getPendingTasks()); + } + catch (Exception e) { + LOG.error(e, "Error fetching pending tasks"); + } + } + return items; + } + + @Override + public Collection getKnownTasks() + { + final Collection items = new ArrayList<>(); + for (TaskRunner runner : runnerMap.values()) { + try { + items.addAll(runner.getKnownTasks()); + } + catch (Exception e) { + LOG.error(e, "Error fetching known tasks"); + } + } + return items; + } + + @Override + public Optional getScalingStats() + { + return Optional.fromNullable(managementStrategy.getStats()); + } + + @Override + public void start() + { + managementStrategy.startManagement(null); + } + + @Override + public void stop() + { + managementStrategy.stopManagement(); + scheduledExecutorService.shutdown(); + } + + @Override + public Optional streamTaskLog(String taskid, long offset) throws IOException + { + // TODO: smarter + for (TaskRunner runner : runnerMap.values()) { + if (!(runner instanceof TaskLogStreamer)) { + LOG.debug("[%s] is not a task log streamer", runner.getClass().getCanonicalName()); + continue; + } + try { + Optional maybeLog = ((TaskLogStreamer) runner).streamTaskLog(taskid, offset); + if (maybeLog.isPresent()) { + return maybeLog; + } + } + catch (Exception e) { + LOG.error(e, "Error fetching log for [%s]", runner); + } + } + LOG.info("Could not find any runners who knew about [%s]", taskid); + return Optional.absent(); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java new file mode 100644 index 000000000000..7febb8cf0574 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java @@ -0,0 +1,47 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.base.Supplier; +import com.google.inject.Inject; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import io.druid.indexing.overlord.routing.TierRouteConfig; + +public class TierRoutingTaskRunnerFactory implements TaskRunnerFactory +{ + private final ScheduledExecutorFactory factory; + private final Supplier routingConfig; + + @Inject + public TierRoutingTaskRunnerFactory( + Supplier routingConfig, + ScheduledExecutorFactory factory + ) + { + this.routingConfig = routingConfig; + this.factory = factory; + } + + @Override + public TierRoutingTaskRunner build() + { + return new TierRoutingTaskRunner(routingConfig, factory); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierTaskDiscovery.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierTaskDiscovery.java new file mode 100644 index 000000000000..a44816836fa5 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierTaskDiscovery.java @@ -0,0 +1,131 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import com.metamx.common.logger.Logger; +import io.druid.guice.annotations.Json; +import io.druid.indexing.overlord.config.TierForkZkConfig; +import io.druid.server.DruidNode; +import org.apache.curator.framework.CuratorFramework; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TierTaskDiscovery +{ + private static final Logger log = new Logger(TierTaskDiscovery.class); + private final CuratorFramework cf; + private final TierForkZkConfig tierForkZkConfig; + private final ObjectMapper mapper; + private final String zkPath; + + @Inject + public TierTaskDiscovery( + TierForkZkConfig tierForkZkConfig, + CuratorFramework cf, + @Json ObjectMapper mapper + ) + { + this.tierForkZkConfig = tierForkZkConfig; + this.cf = cf; + this.mapper = mapper; + this.zkPath = tierForkZkConfig.getTierTaskIDPath(); + } + + public List getTaskIDs() + { + try { + return cf.getChildren().forPath(zkPath); + } + catch (org.apache.zookeeper.KeeperException.NoNodeException e) { + log.info("No node at [%s] for task ids", zkPath); + return ImmutableList.of(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public Optional getNodeForTask(String taskId) + { + final String path = tierForkZkConfig.getTierTaskIDPath(taskId); + try { + final byte[] data = cf.getData().decompressed().forPath(path); + try { + return Optional.of(mapper.readValue(data, DruidNode.class)); + } + catch (IOException e) { + log.warn(e, "Error reading data from [%s]", path); + return Optional.absent(); + } + } + catch (org.apache.zookeeper.KeeperException.NoNodeException e) { + return Optional.absent(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public Map getTasks() + { + final Map map = new HashMap<>(); + try { + for (String taskId : cf.getChildren().forPath(tierForkZkConfig.getTierTaskIDPath())) { + final String child = tierForkZkConfig.getTierTaskIDPath(taskId); + log.debug("Checking [%s]", child); + try { + final byte[] data = cf.getData().decompressed().forPath(child); + if (data == null) { + log.debug("Null data at [%s]", child); + continue; + } + final DruidNode node = mapper.readValue(data, DruidNode.class); + map.put(taskId, node); + } + catch (org.apache.zookeeper.KeeperException.NoNodeException e) { + log.warn("Node vanished at [%s]", child); // Skip + } + catch (IOException e) { + log.error(e, "Failed to parse node data for [%s] at [%s]", taskId, child); + } + catch (Exception e) { + log.error(e, "Error fetching data for node [%s]", child); + } + } + } + catch (org.apache.zookeeper.KeeperException.NoNodeException e) { + log.debug("No node at [%s]", zkPath); + return ImmutableMap.of(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + return ImmutableMap.copyOf(map); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java new file mode 100644 index 000000000000..1bb323d35b5e --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java @@ -0,0 +1,245 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.autoscaling; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.base.Supplier; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TierRoutingTaskRunner; +import io.druid.indexing.overlord.routing.TierRouteConfig; +import io.druid.indexing.overlord.routing.TierTaskRunnerFactory; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +public class TierRoutingManagementStrategy implements ResourceManagementStrategy +{ + private static final Logger LOG = new Logger(TierRoutingManagementStrategy.class); + public static final String ROUTING_TARGET_CONTEXT_KEY = "io.druid.index.tier.target"; + public static final String DEFAULT_ROUTE = "__default"; + private final Supplier configSupplier; + private final ConcurrentMap runnerMap; + private final AtomicBoolean started = new AtomicBoolean(false); + private final ScheduledExecutorService managementExecutorService; + private final AtomicLong numberOfUpdates = new AtomicLong(0L); + + public TierRoutingManagementStrategy( + ConcurrentMap runnerMap, + Supplier configSupplier, + ScheduledExecutorService managementExecutorService + ) + { + this.runnerMap = runnerMap; + this.configSupplier = configSupplier; + this.managementExecutorService = managementExecutorService; + } + + @Override + // State is communicated via configSupplier and runnerMap + public synchronized void startManagement(TierRoutingTaskRunner unused) + { + if (!started.compareAndSet(false, true)) { + throw new ISE("Already started"); + } + if (managementExecutorService.isShutdown()) { + started.set(false); + throw new ISE("Already stopped"); + } + managementExecutorService.scheduleWithFixedDelay( + new Runnable() + { + final AtomicReference priorConfig = new AtomicReference<>(null); + + @Override + public void run() + { + try { + // Local management monitors for config changes. + final TierRouteConfig config = configSupplier.get(); + if (config == null) { + throw new ISE("No config found"); + } + + final TierRouteConfig prior = priorConfig.get(); + if (prior == config) { + LOG.debug("No change in config since last check, skipping update"); + return; + } + + if (!priorConfig.compareAndSet(prior, config)) { + LOG.warn( + "Tier routing config was updated in a racy way... leaving config [%s] and skipping update", + prior + ); + return; + } + + for (String tier : config.getTiers()) { + if (runnerMap.containsKey(tier)) { + LOG.debug("Tier [%s] already in map", tier); + continue; + } + final TierTaskRunnerFactory runnerFactory = config.getRouteFactory(tier); + final TaskRunner runner = runnerFactory.build(); + if (runnerMap.putIfAbsent(tier, runner) != null) { + LOG.warn("Tier [%s] lost a race condition, ignoring runner already in map", tier); + continue; + } + try { + synchronized (TierRoutingManagementStrategy.this) { + if (started.get()) { + runner.start(); + } else { + LOG.warn("Tier [%s] trying to start after shutdown", tier); + if (!runnerMap.remove(tier, runner)) { + // This shouldn't happen, but is here as a super safeguard + LOG.warn("Someone else will have to cleanup the runner for tier [%s], they won a race", tier); + } + } + } + } + catch (Exception e) { + LOG.error(e, "Error starting tier [%s], continuing", tier); + } + } + // TODO: what about tiers that vanish from config? I'm inclined to leave them running in case the vanishing was an error + // Restarting JVM should take care of such a case + } + catch (Exception e) { + LOG.error(e, "Tier routing management encountered exception. Trying again"); + } + finally { + // Used in unit tests + synchronized (numberOfUpdates) { + numberOfUpdates.incrementAndGet(); + numberOfUpdates.notifyAll(); + } + } + } + }, + 0, + 10, // TODO: make this configurable + TimeUnit.SECONDS + ); + } + + @VisibleForTesting + void waitForUpdate() throws InterruptedException + { + final long startingUpdates = numberOfUpdates.get(); + while (startingUpdates == numberOfUpdates.get()) { + synchronized (numberOfUpdates) { + numberOfUpdates.wait(); + } + } + } + + @Override + public synchronized void stopManagement() + { + if (!started.compareAndSet(true, false)) { + LOG.warn("Ignoring repeated stop request"); + return; + } + managementExecutorService.shutdown(); + try { + if (!managementExecutorService.awaitTermination(10, TimeUnit.SECONDS)) { + LOG.warn("Could not shut down all tasks. Continuing anyways"); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error(e, "Interrupted"); + } + for (String tier : runnerMap.keySet()) { + final TaskRunner runner = runnerMap.get(tier); + if (runner == null) { + LOG.warn("Race condition for tier [%s]", tier); + continue; + } + try { + runner.stop(); + } + catch (Exception e) { + LOG.error(e, "Error shutting down runner for tier [%s]", tier); + } + runnerMap.remove(tier); + } + } + + @Override + public synchronized ScalingStats getStats() + { + final ScalingStats stats = new ScalingStats(0); + final AtomicBoolean foundSomething = new AtomicBoolean(false); + stats.addAllEvents(ImmutableList.copyOf( + FluentIterable + .from(runnerMap.values()) + .transformAndConcat(new Function>() + { + @Nullable + @Override + public List apply(@Nullable TaskRunner runner) + { + if (runner == null) { + return ImmutableList.of(); + } + final Optional stats = runner.getScalingStats(); + if (stats.isPresent()) { + foundSomething.set(true); + return stats.get().toList(); + } else { + return ImmutableList.of(); + } + } + }) + )); + return foundSomething.get() ? stats : null; + } + + public TaskRunner getRunner(Task task) + { + final Object tierobj = task.getContextValue(ROUTING_TARGET_CONTEXT_KEY); + final String tier; + if (tierobj == null) { + LOG.debug("No route context found for task [%s]. Using default [%s]", task.getId(), DEFAULT_ROUTE); + tier = DEFAULT_ROUTE; + } else { + tier = tierobj.toString(); + } + + LOG.info("Using tier [%s] for task [%s]", tier, task.getId()); + + return runnerMap.get(tier); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java new file mode 100644 index 000000000000..9fe379361713 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java @@ -0,0 +1,55 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.config; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.indexing.overlord.TierTaskDiscovery; +import io.druid.indexing.overlord.routing.TierRouteConfig; +import io.druid.indexing.overlord.routing.TierTaskRunnerFactory; +import io.druid.initialization.DruidModule; + +import java.util.List; + +public class TierConfigModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule("TierConfigModule").registerSubtypes( + TierRouteConfig.class, + TierTaskRunnerFactory.class + ) + ); + } + + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.indexer.runner", TierLocalTaskRunnerConfig.class); + JsonConfigProvider.bind(binder, "druid.zk.paths", TierForkZkConfig.class); + binder.bind(TierTaskDiscovery.class).in(LazySingleton.class); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierForkZkConfig.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierForkZkConfig.java new file mode 100644 index 000000000000..c7306425f282 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierForkZkConfig.java @@ -0,0 +1,74 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.config; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import io.druid.server.initialization.ZkPathsConfig; +import org.apache.curator.utils.ZKPaths; + +public class TierForkZkConfig +{ + @JsonCreator + public TierForkZkConfig( + @JacksonInject ZkPathsConfig zkPathsConfig, + @JsonProperty("tierLeaderBasePath") String tierLeaderBasePath, + @JsonProperty("tierTaskIDPath") String tierTaskIDPath + ) + { + this.tierLeaderBasePath = tierLeaderBasePath; + this.tierTaskIDPath = tierTaskIDPath; + this.zkPathsConfig = zkPathsConfig; + } + + @JsonProperty + public String tierTaskIDPath = null; + + @JsonProperty + public String tierLeaderBasePath = null; + + @JsonIgnore + @JacksonInject + public ZkPathsConfig zkPathsConfig = new ZkPathsConfig(); + + public String getTierTaskIDPath() + { + return tierTaskIDPath != null ? tierTaskIDPath : zkPathsConfig.defaultPath("tierTasks"); + } + + public String getTierLeaderBasePath() + { + return tierLeaderBasePath != null ? tierLeaderBasePath : zkPathsConfig.defaultPath("tierLeaders"); + } + + public String getTierLeaderPath(String tier) + { + return ZKPaths.makePath(getTierLeaderBasePath(), Preconditions.checkNotNull(Strings.emptyToNull(tier), "tier")); + } + + public String getTierTaskIDPath(String taskId) + { + return ZKPaths.makePath(getTierTaskIDPath(), taskId); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierLocalTaskRunnerConfig.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierLocalTaskRunnerConfig.java new file mode 100644 index 000000000000..8242506ef263 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierLocalTaskRunnerConfig.java @@ -0,0 +1,112 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import javax.validation.constraints.Min; + +public class TierLocalTaskRunnerConfig extends ForkingTaskRunnerConfig +{ + /** + * This is the time (in ms) that the forking task runner should allow the task to softly shutdown before trying to forcibly kill it. + */ + @JsonProperty + @Min(0) + private long softShutdownTimeLimit = 30_000L; + + /** + * The time (in ms) to wait to receive a heartbeat before terminating + */ + @JsonProperty + @Min(0) + private long heartbeatTimeLimit = 300_000L; + + /** + * Time (in ms) to wait for network timeout when sending heartbeats to tasks. + * WARNING: Tasks are polled serially, and only local tasks are polled by this runner, so this value should be + * low and significantly lower than heartbeatTimeLimit / maxHeartbeatRetries + * Failure to send a heartbeat will simply log the error and wait until the next round of heartbeat attempts. + */ + @JsonProperty + @Min(0) + private long heartbeatLocalNetworkTimeout = 100L; + + /** + * Maximum count of retries to send a heartbeat to a local worker. Specifically, the delay between post attempts + * follows this formula: heartbeatTimeLimit / maxHeartbeatRetries + */ + + @JsonProperty + @Min(0) + private long maxHeartbeatRetries = 10L; + + public TierLocalTaskRunnerConfig setMaxHeartbeatRetries(long retries) + { + Preconditions.checkArgument(retries >= 0, "retries too small"); + maxHeartbeatRetries = retries; + return this; + } + + public long getMaxHeartbeatRetries() + { + return maxHeartbeatRetries; + } + + public long getDelayBetweenHeartbeatBatches() + { + return getHeartbeatTimeLimit() / getMaxHeartbeatRetries(); + } + + public TierLocalTaskRunnerConfig setHeartbeatLocalNetworkTimeout(long timeout) + { + Preconditions.checkArgument(timeout >= 0, "timeout too small"); + heartbeatLocalNetworkTimeout = timeout; + return this; + } + + public long getHeartbeatLocalNetworkTimeout() + { + return heartbeatLocalNetworkTimeout; + } + + public TierLocalTaskRunnerConfig setSoftShutdownTimeLimit(long softShutdownTimeLimit) + { + this.softShutdownTimeLimit = softShutdownTimeLimit; + return this; + } + + public long getSoftShutdownTimeLimit() + { + return softShutdownTimeLimit; + } + + public TierLocalTaskRunnerConfig setHeartbeatTimeLimit(long heartbeatTimeLimit) + { + this.heartbeatTimeLimit = heartbeatTimeLimit; + return this; + } + + public long getHeartbeatTimeLimit() + { + return heartbeatTimeLimit; + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandMonitor.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandMonitor.java new file mode 100644 index 000000000000..52c44190d050 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandMonitor.java @@ -0,0 +1,203 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + +import com.google.common.base.Throwables; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import io.druid.concurrent.Execs; +import io.druid.indexing.overlord.TierLocalTaskRunner; +import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; + +public class DeadhandMonitor +{ + private static final Logger log = new Logger(DeadhandMonitor.class); + private final AtomicBoolean started = new AtomicBoolean(false); + private final DeadhandResource resource; + private final File deadhandFile; + private final ListeningExecutorService watchdogService = MoreExecutors.listeningDecorator(Execs.multiThreaded( + 2, + "deadhandWatchdog-%s" + )); + private final long timeout; + + @Inject + public DeadhandMonitor( + final DeadhandResource resource, + final TierLocalTaskRunnerConfig tierLocalTaskRunnerConfig + ) + { + this(resource, tierLocalTaskRunnerConfig, new File(TierLocalTaskRunner.DEADHAND_FILE_NAME)); + } + + public DeadhandMonitor( + final DeadhandResource resource, + final TierLocalTaskRunnerConfig tierLocalTaskRunnerConfig, + final File deadhandFile + ) + { + this.resource = resource; + this.timeout = tierLocalTaskRunnerConfig.getHeartbeatTimeLimit(); + this.deadhandFile = deadhandFile; + } + + void exit() + { + System.exit(0xDEAD); + } + + @LifecycleStart + public void start() + { + synchronized (started) { + if (watchdogService.isShutdown()) { + throw new ISE("Already stopped"); + } + ListenableFuture future = watchdogService.submit( + new Runnable() + { + @Override + public void run() + { + boolean shouldExit = true; + final Path deadhandParentPath = deadhandFile.getAbsoluteFile().toPath().getParent(); + try (final WatchService watchService = deadhandParentPath.getFileSystem().newWatchService()) { + deadhandParentPath.register(watchService, StandardWatchEventKinds.ENTRY_DELETE); + log.info("Monitoring [%s] for shutdown", deadhandFile); + while (deadhandFile.exists()) { + watchService.poll(10, TimeUnit.SECONDS); + } + log.warn("[%s] vanished! exiting", deadhandFile); + } + catch (IOException e) { + log.error(e, "Could not register deadhand watchdog!"); + } + catch (InterruptedException e) { + shouldExit = false; + log.info("Interrupted while watching deadhand file"); + } + if (shouldExit) { + exit(); + } + } + } + ); + Futures.addCallback(future, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + log.debug("deadhand file watch finished"); + } + + @Override + public void onFailure(Throwable t) + { + if (t.getCause() instanceof InterruptedException) { + log.debug("Deadhand file watch interrupted"); + } else { + log.error(t, "Failed deadhand file watch"); + } + } + }); + future = watchdogService.submit( + new Runnable() + { + @Override + public void run() + { + while (!Thread.currentThread().isInterrupted()) { + try { + resource.waitForHeartbeat(timeout); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.warn("Interrupted, stopping deadhand watchdog"); + throw Throwables.propagate(e); + } + catch (TimeoutException e) { + boolean exiting = false; + synchronized (started) { + if (started.get()) { + log.error(e, "Timeout reached. I shall ride eternal, shiny and chrome!"); + exiting = true; + } else { + log.warn("Timeout but not started"); + } + } + if (exiting) { + // Outside of synchronized block + exit(); + } + } + } + } + } + ); + Futures.addCallback(future, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + log.debug("Resource watch finished"); + } + + @Override + public void onFailure(Throwable t) + { + if (t.getCause() instanceof InterruptedException) { + log.debug("Deadhand resource watcher interrupted"); + } else { + log.error(t, "Error watching deadhand resource"); + } + } + }); + resource.refresh(); + started.set(true); + } + } + + @LifecycleStop + public void stop() + { + synchronized (started) { + started.set(false); + watchdogService.shutdownNow(); + } + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandResource.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandResource.java new file mode 100644 index 000000000000..82664b8b4de6 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandResource.java @@ -0,0 +1,89 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + + +import com.metamx.common.logger.Logger; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.Response; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Quite simply, an endpoint endpoint with an accompanying method which will wait for a determined amount of time. + * It is intended to be used in a watchdog or deadhand system whereby if the timeout is exhausted then something + * happens. + */ +@Path(DeadhandResource.DEADHAND_PATH) +public class DeadhandResource +{ + public static final String DEADHAND_PATH = "/deadhand/stall"; + private static final Logger log = new Logger(DeadhandResource.class); + private final AtomicLong heartbeatDetector = new AtomicLong(0L); + + @POST + public Response doHeartbeat(@Context final HttpServletRequest req) + { + log.info("Received stall from [%s]", req != null ? req.getRemoteAddr() : "unknown"); + refresh(); + return Response.ok().build(); + } + + void refresh() + { + synchronized (heartbeatDetector) { + heartbeatDetector.incrementAndGet(); + heartbeatDetector.notifyAll(); + } + } + + public void waitForHeartbeat(long timeout) throws InterruptedException, TimeoutException + { + final long start = System.currentTimeMillis(); + synchronized (heartbeatDetector) { + final long pre = heartbeatDetector.get(); + do { + final long t = timeout - (System.currentTimeMillis() - start); + if (t > 0) { + heartbeatDetector.wait(t); + } + if (heartbeatDetector.get() != pre) { + log.debug("Heartbeat heard"); + return; + } + // See docs about wait regarding spurious wakeup + } while (System.currentTimeMillis() - start < timeout); + if (pre == heartbeatDetector.get()) { + throw new TimeoutException(String.format("Not heard within %d ms", timeout)); + } + log.debug("Timeout, but heartbeat heard anyways. Phew! that was close"); + } + } + + // Protected for tests + protected long getHeartbeatCount() + { + return heartbeatDetector.get(); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/ShutdownCleanlyResource.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/ShutdownCleanlyResource.java new file mode 100644 index 000000000000..e177547a1ebd --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/ShutdownCleanlyResource.java @@ -0,0 +1,58 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + +import io.druid.concurrent.Execs; + +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.core.Response; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +@Path(ShutdownCleanlyResource.PATH) +public class ShutdownCleanlyResource +{ + public static final String PATH = "/druid/tier/v1/shutdown"; + + @POST + public Response shutdown() + { + final ScheduledExecutorService shutdownService = Execs.scheduledSingleThreaded("TierShutterDowner"); + shutdownService.schedule( + new Runnable() + { + @Override + public void run() + { + exit(); + } + }, + 1, TimeUnit.SECONDS + ); + return Response.status(Response.Status.ACCEPTED).build(); + } + + // Can be overridden in tests + protected void exit() + { + System.exit(0); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TaskLogResource.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TaskLogResource.java new file mode 100644 index 000000000000..7a05a785a729 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TaskLogResource.java @@ -0,0 +1,87 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + +import com.google.common.base.Preconditions; +import com.metamx.common.logger.Logger; +import io.druid.indexing.common.tasklogs.LogUtils; +import io.druid.indexing.overlord.TierLocalTaskRunner; +import io.druid.server.DruidNode; + +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; + +@Path(TaskLogResource.PATH) +public class TaskLogResource +{ + private static final Logger log = new Logger(TaskLogResource.class); + public static final String PATH = "/druid/worker/v1/task/log"; + public static final String OFFSET_PARAM = "offset"; + + @GET + @Produces(MediaType.TEXT_PLAIN) + public Response getLog( + @QueryParam(OFFSET_PARAM) @DefaultValue("0") long offset + ) + { + final File logFile = new File(TierLocalTaskRunner.LOG_FILE_NAME); + + if (!logFile.exists()) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + if (!logFile.canRead()) { + return Response.status(Response.Status.FORBIDDEN).build(); + } + + try { + return Response.ok(LogUtils.streamFile(logFile, offset)).build(); + } + catch (FileNotFoundException e) { + log.wtf(e, "File [%s] not found, but was found!?", logFile); + return Response.status(Response.Status.GONE).build(); + } + catch (IOException e) { + log.error(e, "Error fetching log file"); + return Response.serverError().build(); + } + } + + public static URL buildURL(DruidNode targetNode, long offset) throws MalformedURLException + { + Preconditions.checkArgument(offset >= 0, "offset must be >= 0"); + return new URL( + "http", + targetNode.getHost(), + targetNode.getPort(), + String.format("%s?%s=%d", PATH, OFFSET_PARAM, offset) + ); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TierRunningCheckResource.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TierRunningCheckResource.java new file mode 100644 index 000000000000..e7afcc9d3b8a --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TierRunningCheckResource.java @@ -0,0 +1,37 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.core.Response; + +@Path(TierRunningCheckResource.PATH) +public class TierRunningCheckResource +{ + public static final String PATH = "/druid/tier/v1/running"; + + @GET + public Response get() + { + return Response.ok().build(); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/ForkingTaskRunnerTierFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/ForkingTaskRunnerTierFactory.java new file mode 100644 index 000000000000..e27988371803 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/ForkingTaskRunnerTierFactory.java @@ -0,0 +1,111 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.guice.annotations.Self; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.overlord.ForkingTaskRunnerFactory; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; +import io.druid.indexing.worker.config.WorkerConfig; +import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogPusher; + +import java.util.Properties; + +public class ForkingTaskRunnerTierFactory implements TierTaskRunnerFactory +{ + private final ForkingTaskRunnerConfig forkingTaskRunnerConfig; + private final TaskConfig taskConfig; + private final WorkerConfig workerConfig; + private final Properties properties; + private final ObjectMapper jsonMapper; + private final TaskLogPusher persistentTaskLogs; + private final DruidNode node; + + @JsonCreator + public ForkingTaskRunnerTierFactory( + @JsonProperty + final ForkingTaskRunnerConfig forkingTaskRunnerConfig, + @JsonProperty + final TaskConfig taskConfig, + @JsonProperty + final WorkerConfig workerConfig, + @JsonProperty + final Properties properties, + @JacksonInject + final ObjectMapper jsonMapper, + @JacksonInject + final TaskLogPusher persistentTaskLogs, + @JacksonInject + @Self DruidNode node + ) + { + this.forkingTaskRunnerConfig = forkingTaskRunnerConfig; + this.taskConfig = taskConfig; + this.workerConfig = workerConfig; + this.properties = properties; + this.jsonMapper = jsonMapper; + this.persistentTaskLogs = persistentTaskLogs; + this.node = node; + } + + @Override + public TaskRunner build() + { + return new ForkingTaskRunnerFactory( + forkingTaskRunnerConfig, + taskConfig, + workerConfig, + properties, + jsonMapper, + persistentTaskLogs, + node + ).build(); + } + + @JsonProperty + public ForkingTaskRunnerConfig getForkingTaskRunnerConfig() + { + return forkingTaskRunnerConfig; + } + + @JsonProperty + public TaskConfig getTaskConfig() + { + return taskConfig; + } + + @JsonProperty + public WorkerConfig getWorkerConfig() + { + return workerConfig; + } + + @JsonProperty + public Properties getProperties() + { + return properties; + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactory.java new file mode 100644 index 000000000000..935e81188119 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactory.java @@ -0,0 +1,133 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.http.client.HttpClient; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Json; +import io.druid.indexing.overlord.RemoteTaskRunnerFactory; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.autoscaling.NoopResourceManagementStrategy; +import io.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerResourceManagementConfig; +import io.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerResourceManagementStrategy; +import io.druid.indexing.overlord.autoscaling.ResourceManagementSchedulerConfig; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import io.druid.server.initialization.IndexerZkConfig; +import org.apache.curator.framework.CuratorFramework; + +public class RemoteTaskRunnerTierFactory implements TierTaskRunnerFactory +{ + final CuratorFramework curator; + final RemoteTaskRunnerConfig remoteTaskRunnerConfig; + final IndexerZkConfig zkPaths; + final ObjectMapper jsonMapper; + final HttpClient httpClient; + final Supplier workerConfigRef; + final ScheduledExecutorFactory factory; + final PendingTaskBasedWorkerResourceManagementConfig pendingTaskBasedWorkerResourceManagementConfig; + final ResourceManagementSchedulerConfig resourceManagementSchedulerConfig; + + @JsonCreator + public RemoteTaskRunnerTierFactory( + @JsonProperty + final RemoteTaskRunnerConfig remoteTaskRunnerConfig, + @JsonProperty + final PendingTaskBasedWorkerResourceManagementConfig pendingTaskBasedWorkerResourceManagementConfig, + // This is part of why this is not compatible with the tiering methodology. + @JacksonInject + final Supplier workerConfigRef, + @JacksonInject + final CuratorFramework curator, + @JacksonInject + final IndexerZkConfig zkPaths, + @JacksonInject + @Json + final ObjectMapper jsonMapper, + @JacksonInject + @Global final HttpClient httpClient, + @JacksonInject + final ScheduledExecutorFactory factory, + @JacksonInject + final ResourceManagementSchedulerConfig resourceManagementSchedulerConfig + ) + { + this.remoteTaskRunnerConfig = remoteTaskRunnerConfig; + this.workerConfigRef = workerConfigRef; + this.pendingTaskBasedWorkerResourceManagementConfig = pendingTaskBasedWorkerResourceManagementConfig; + this.curator = curator; + this.zkPaths = zkPaths; + this.jsonMapper = jsonMapper; + this.httpClient = httpClient; + this.factory = factory; + this.resourceManagementSchedulerConfig = resourceManagementSchedulerConfig; + } + + @Override + public TaskRunner build() + { + return new RemoteTaskRunnerFactory( + curator, + remoteTaskRunnerConfig, + zkPaths, + jsonMapper, + httpClient, + workerConfigRef, + factory, + resourceManagementSchedulerConfig, + resourceManagementSchedulerConfig.isDoAutoscale() ? new PendingTaskBasedWorkerResourceManagementStrategy( + pendingTaskBasedWorkerResourceManagementConfig, + workerConfigRef, + resourceManagementSchedulerConfig, + factory + ) : new NoopResourceManagementStrategy() + ).build(); + } + + @JsonProperty + public RemoteTaskRunnerConfig getRemoteTaskRunnerConfig() + { + return remoteTaskRunnerConfig; + } + + @JsonProperty + public IndexerZkConfig getZkPaths() + { + return zkPaths; + } + + @JsonProperty + public PendingTaskBasedWorkerResourceManagementConfig getPendingTaskBasedWorkerResourceManagementConfig() + { + return pendingTaskBasedWorkerResourceManagementConfig; + } + + @JsonProperty + public ResourceManagementSchedulerConfig getResourceManagementSchedulerConfig() + { + return resourceManagementSchedulerConfig; + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResource.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResource.java new file mode 100644 index 000000000000..04b800e9b7c3 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResource.java @@ -0,0 +1,77 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import com.google.inject.name.Named; +import io.druid.indexing.common.TaskStatus; +import io.druid.server.DruidNode; + +import javax.validation.constraints.NotNull; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.net.MalformedURLException; +import java.net.URL; + +@Path(TaskStatusPostToLeaderListenerResource.PATH) +public class TaskStatusPostToLeaderListenerResource +{ + private final TaskStatusReporter upstreamReporter; + + @Inject + public TaskStatusPostToLeaderListenerResource( + @Named(TaskStatusReporterModule.UPSTREAM_TASK_REPORTER_NAME) TaskStatusReporter upstreamReporter + ) + { + this.upstreamReporter = upstreamReporter; + } + + public static final String PATH = "/druid/indexer/tier/v1/report"; + + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response doPost(@NotNull TaskStatus status) + { + try { + Preconditions.checkNotNull(status, "status"); + if (upstreamReporter.reportStatus(status)) { + return Response.status(Response.Status.ACCEPTED).entity(status).build(); + } + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity(status).build(); + } + catch (RuntimeException e) { + return Response.serverError() + .entity(ImmutableMap.of("error", e.getMessage() == null ? "null" : e.getMessage())) + .build(); + } + } + + public static URL makeReportUrl(DruidNode node) throws MalformedURLException + { + return new URL("http", node.getHost(), node.getPort(), PATH); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java new file mode 100644 index 000000000000..7e981905457e --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java @@ -0,0 +1,120 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import com.google.inject.Inject; +import com.google.inject.name.Named; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.guice.annotations.Global; +import io.druid.indexing.common.TaskStatus; +import io.druid.server.DruidNode; +import org.apache.curator.x.discovery.ServiceDiscovery; +import org.apache.curator.x.discovery.ServiceInstance; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.Duration; + +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Collection; +import java.util.concurrent.ExecutionException; + +/** + * A TaskStatusReporter which looks for a tier and reports with a POST. + * It assumes the service will have a redirect if it is not the current leader. + */ +public class TaskStatusPostToLeaderReporter implements TaskStatusReporter +{ + private static final Logger log = new Logger(TaskStatusPostToLeaderReporter.class); + private final HttpClient httpClient; + private final ServiceDiscovery discovery; + private final String upstreamService; + + @Inject + public TaskStatusPostToLeaderReporter( + @Global HttpClient httpClient, + ServiceDiscovery discovery, + @Named(TaskStatusReporterModule.UPSTREAM_SERVICE_NAME_CONSTANT_KEY) String upstreamService + ) + { + this.httpClient = httpClient; + this.discovery = discovery; + this.upstreamService = upstreamService; + } + + @Override + public boolean reportStatus(TaskStatus status) + { + final Collection> overlords; + try { + overlords = discovery.queryForInstances(upstreamService); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + if (overlords.isEmpty()) { + throw new ISE("No overlords found for service [%s]", upstreamService); + } + + final ServiceInstance overlord = Iterables.getFirst(overlords, null); + if (overlord == null) { + throw new ISE("Overlords not empty but had no entries?"); + } + final DruidNode node = new DruidNode("ignored", overlord.getAddress(), overlord.getPort()); + try { + final URL url = TaskStatusPostToLeaderListenerResource.makeReportUrl(node); + final StatusResponseHolder response = httpClient.go( + new Request( + HttpMethod.POST, + url + ), new StatusResponseHandler(Charsets.UTF_8), + Duration.millis(60_000) // TODO: make this configurable + ).get(); + log.debug("Received [%s] for reporting status [%s] to [%s]", response, status, url); + if (response.getStatus().equals(HttpResponseStatus.ACCEPTED)) { + return true; + } + if (response.getStatus().equals(HttpResponseStatus.SERVICE_UNAVAILABLE)) { + return false; + } + throw new ISE("Unknown response [%s] when submitting [%s] to [%s]", response, status, url); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException | MalformedURLException e) { + throw Throwables.propagate(e); + } + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporter.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporter.java new file mode 100644 index 000000000000..ed75e0fe1bdd --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporter.java @@ -0,0 +1,44 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import io.druid.indexing.common.TaskStatus; + +/** + * A TaskStatusReporter is responsible for pushing task results back up the chain of custody. + * It is expected that results reporting is idempotent. + */ + +public interface TaskStatusReporter +{ + /** + * Report the status of the task. The intended order of status reports is not specified. Any TaskStatusReporter + * is expected to return TRUE if the status reports do not occur in order. + * + * @param status The TaskStatus to report + * + * @return True if the report succeeded. Specifically, a result of true indicates that the receiving end has heard + * the report and further attempts to report the same status will be idempotent and not necessary. A value of true + * does NOT indicate that the status has ben SET for the task, but rather that the status notification has been + * handled in a highly available way. A value of FALSE indicates that something isn't right and the caller should + * try again, preferably checking making sure that this method is called on the leader + */ + boolean reportStatus(TaskStatus status); +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporterModule.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporterModule.java new file mode 100644 index 000000000000..7476d7be5773 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporterModule.java @@ -0,0 +1,171 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.Module; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.name.Named; +import com.google.inject.name.Names; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import io.druid.client.indexing.IndexingServiceSelectorConfig; +import io.druid.guice.JacksonConfigProvider; +import io.druid.guice.Jerseys; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.guice.PolyBind; +import io.druid.guice.annotations.Global; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TaskRunnerFactory; +import io.druid.indexing.overlord.TierRoutingTaskRunnerFactory; +import io.druid.initialization.DruidModule; +import org.apache.curator.x.discovery.ServiceDiscovery; + +import java.util.List; + +public class TaskStatusReporterModule implements DruidModule +{ + private static final Logger LOG = new Logger(TaskStatusReporterModule.class); + public static final String UPSTREAM_SERVICE_NAME_CONSTANT_KEY = "upstreamServiceName"; + public static final String UPSTREAM_TASK_REPORTER_NAME = "upstream"; + public static final String UPSTREAM_PROPERTY_KEY = "io.druid.index.tier.upstreamServiceName"; + public static final String POLYBIND_ROUTING_KEY = "routing"; + public static final String ROUTING_CONFIG_KEY = "druid.tier.routing.config"; + + @Override + public List getJacksonModules() + { + return ImmutableList.of(); + } + + @Override + public void configure(Binder binder) + { + PolyBind.optionBinder( + binder, + Key.get(TaskRunnerFactory.class) + ).addBinding(POLYBIND_ROUTING_KEY).to(TierRoutingTaskRunnerFactory.class); + + Jerseys.addResource(binder, TaskStatusPostToLeaderListenerResource.class); + + binder.bind(Key.get(TaskStatusReporter.class, Names.named(UPSTREAM_TASK_REPORTER_NAME))) + .to(DynamicUpstreamReporter.class) + .in(LazySingleton.class); + JsonConfigProvider.bind(binder, UPSTREAM_PROPERTY_KEY, UpstreamNameHolder.class); + binder.bind(Key.get(String.class, Names.named(UPSTREAM_SERVICE_NAME_CONSTANT_KEY))) + .toProvider(UpstreamNameHolderProvider.class); + JacksonConfigProvider.bind(binder, ROUTING_CONFIG_KEY, TierRouteConfig.class, null); + } + + public static class UpstreamNameHolder + { + @JsonProperty + String upstreamServiceName = IndexingServiceSelectorConfig.DEFAULT_SERVICE_NAME; + + public String getUpstreamServiceName() + { + return upstreamServiceName; + } + } + + public static class UpstreamNameHolderProvider implements Provider + { + private final UpstreamNameHolder upstreamNameHolder; + + @Inject + public UpstreamNameHolderProvider( + UpstreamNameHolder upstreamNameHolder + ) + { + this.upstreamNameHolder = upstreamNameHolder; + } + + @Override + public String get() + { + return upstreamNameHolder.getUpstreamServiceName(); + } + } + + public static class DynamicUpstreamReporter implements TaskStatusReporter + { + @Inject(optional = true) + private TaskMaster taskMaster = null; + @Inject + private + @Global + HttpClient httpClient = null; + @Inject + private ServiceDiscovery discovery = null; + @Inject + private + @Named(UPSTREAM_SERVICE_NAME_CONSTANT_KEY) + String upstreamService = null; + + public DynamicUpstreamReporter() {} + + public DynamicUpstreamReporter( + TaskMaster taskMaster, + @Global HttpClient httpClient, + ServiceDiscovery discovery, + @Named(UPSTREAM_SERVICE_NAME_CONSTANT_KEY) String upstreamService + ) + { + this.taskMaster = taskMaster; + this.httpClient = httpClient; + this.discovery = discovery; + this.upstreamService = upstreamService; + } + + @Override + public boolean reportStatus(TaskStatus status) + { + TaskStatusReporter reporter = new TaskStatusPostToLeaderReporter(httpClient, discovery, upstreamService); + if (taskMaster != null) { + final Optional taskRunnerOptional = taskMaster.getTaskRunner(); + if (taskRunnerOptional.isPresent()) { + final TaskRunner runner = taskRunnerOptional.get(); + if (runner instanceof TaskStatusReporter) { + reporter = (TaskStatusReporter) runner; + } else { + LOG.debug( + "Expected [%s] but was [%s]. Trying POST", + TaskStatusReporter.class, + runner.getClass().getCanonicalName() + ); + } + } else { + LOG.debug("No task runner. Trying POST"); + } + } else { + LOG.debug("No task master. Trying POST"); + } + return reporter.reportStatus(status); + } + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierRouteConfig.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierRouteConfig.java new file mode 100644 index 000000000000..90818fdfc37b --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierRouteConfig.java @@ -0,0 +1,62 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; + +import javax.validation.constraints.NotNull; +import java.util.Arrays; +import java.util.Map; +import java.util.Set; + +public class TierRouteConfig +{ + // Fields exposed for unit tests + @JsonProperty("tierMap") + Map> tierMap = ImmutableMap.of(); + + @JacksonInject + ObjectMapper mapper = null; + + public + @NotNull + TierTaskRunnerFactory getRouteFactory(@NotNull String tier) + { + final Map map = tierMap.get(Preconditions.checkNotNull(tier, "tier")); + if (map == null) { + throw new NullPointerException( + String.format( + "No tier found for [%s]. Valid tier are %s", tier, + Arrays.toString(tierMap.keySet().toArray()) + ) + ); + } + return mapper.convertValue(map, TierTaskRunnerFactory.class); + } + + public Set getTiers() + { + return tierMap.keySet(); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierTaskRunnerFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierTaskRunnerFactory.java new file mode 100644 index 000000000000..ef14d0685114 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierTaskRunnerFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.druid.indexing.overlord.ForkingTaskRunnerFactory; +import io.druid.indexing.overlord.RemoteTaskRunnerFactory; +import io.druid.indexing.overlord.TaskRunner; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") + +/** + * A factory for building task runners for Tiering. This differs from the default TaskRunnerFactory because these are + * creatable from JSON, whereas the default ones are only creatable via Guice injection + */ +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = UnknownRouteFactory.TYPE_NAME, value = UnknownRouteFactory.class), + @JsonSubTypes.Type(name = RemoteTaskRunnerFactory.TYPE_NAME, value = RemoteTaskRunnerTierFactory.class), + @JsonSubTypes.Type(name = ForkingTaskRunnerFactory.TYPE_NAME, value = ForkingTaskRunnerTierFactory.class) +}) +public interface TierTaskRunnerFactory +{ + TaskRunner build(); +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/UnknownRouteFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/UnknownRouteFactory.java new file mode 100644 index 000000000000..d63b35c07944 --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/UnknownRouteFactory.java @@ -0,0 +1,35 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.metamx.common.UOE; +import io.druid.indexing.overlord.TaskRunner; + +public class UnknownRouteFactory implements TierTaskRunnerFactory +{ + public final static TierTaskRunnerFactory INSTANCE = new UnknownRouteFactory(); + final static String TYPE_NAME = "__UNKNOWN"; + + @Override + public TaskRunner build() + { + throw new UOE("Cannot launch task. Unknown Route"); + } +} diff --git a/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.cli.CliCommandCreator b/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.cli.CliCommandCreator new file mode 100644 index 000000000000..396616106ce3 --- /dev/null +++ b/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.cli.CliCommandCreator @@ -0,0 +1,20 @@ +# +# Licensed to Metamarkets Group Inc. (Metamarkets) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. Metamarkets licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +io.druid.cli.CliTierCreator diff --git a/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..3a3920c22156 --- /dev/null +++ b/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1,21 @@ +# +# Licensed to Metamarkets Group Inc. (Metamarkets) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. Metamarkets licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +io.druid.indexing.overlord.config.TierConfigModule +io.druid.indexing.overlord.routing.TaskStatusReporterModule diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierCreatorTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierCreatorTest.java new file mode 100644 index 000000000000..67f9b268207f --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierCreatorTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import io.airlift.airline.Cli; +import io.airlift.airline.model.CommandGroupMetadata; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; + +public class CliTierCreatorTest +{ + + @Test + public void testAddCommands() throws Exception + { + final Cli.CliBuilder builder = new Cli.CliBuilder("builder"); + final CliTierCreator creator = new CliTierCreator(); + creator.addCommands(builder); + final Cli cli = builder.build(); + Assert.assertEquals(ImmutableList.of("tier"), Lists.transform( + cli.getMetadata().getCommandGroups(), + new Function() + { + @Nullable + @Override + public String apply(@Nullable CommandGroupMetadata input) + { + return input.getName(); + } + } + )); + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java new file mode 100644 index 000000000000..bc2f83cd3dae --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java @@ -0,0 +1,193 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Binding; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.util.Modules; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.RemoteChatHandler; +import io.druid.guice.annotations.Self; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; +import io.druid.indexing.overlord.resources.DeadhandResource; +import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; +import io.druid.initialization.Initialization; +import io.druid.server.DruidNode; +import io.druid.server.initialization.ServerConfig; +import org.easymock.EasyMock; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.junit.After; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.concurrent.TimeoutException; + +public class CliTierForkTest +{ + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + @Test + public void testGetModules() throws Exception + { + final CliTierFork cliTierFork = new CliTierFork(); + cliTierFork.taskAndStatusFile = ImmutableList.of( + temporaryFolder.newFile().toString(), + temporaryFolder.newFile().toString() + ); + final Injector startupInjector = GuiceInjectors.makeStartupInjector(); + startupInjector.injectMembers(cliTierFork); + + final Injector injector = Initialization.makeInjectorWithModules( + startupInjector, + ImmutableList.of( + Modules.override(cliTierFork.getModules()).with( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null) + ); + JsonConfigProvider.bindInstance( + binder, + Key.get(DruidNode.class, RemoteChatHandler.class), + new DruidNode("test", "localhost", null) + ); + JsonConfigProvider.bindInstance( + binder, + Key.get(ServerConfig.class, RemoteChatHandler.class), + new ServerConfig() + ); + } + }) + ) + ); + final Binding binding = injector.getBinding(ForkAnnouncer.class); + Assert.assertNotNull(binding); + } + + @Test + public void testParentMonitorInputStreamFakerProviderTIMEOUT() throws Exception + { + final TierLocalTaskRunnerConfig config = new TierLocalTaskRunnerConfig(); + final DeadhandResource resource = EasyMock.createStrictMock(DeadhandResource.class); + resource.waitForHeartbeat(EasyMock.anyLong()); + EasyMock.expectLastCall().once().andThrow(new TimeoutException("test exception")).once(); + EasyMock.replay(resource); + final ParentMonitorInputStreamFakerProvider provider = new ParentMonitorInputStreamFakerProvider( + resource, + config + ); + try (final InputStream stream = provider.get()) { + Assert.assertEquals(0, stream.read()); + Assert.assertEquals(-1, stream.read()); + } + EasyMock.verify(resource); + } + + @Test + public void testParentMonitorInputStreamFakerProviderINTERRUPT() throws Exception + { + expectedEx.expect(IOException.class); + final TierLocalTaskRunnerConfig config = new TierLocalTaskRunnerConfig(); + final DeadhandResource resource = EasyMock.createStrictMock(DeadhandResource.class); + resource.waitForHeartbeat(EasyMock.anyLong()); + EasyMock.expectLastCall().andThrow(new InterruptedException("test exception")).once(); + EasyMock.replay(resource); + final ParentMonitorInputStreamFakerProvider provider = new ParentMonitorInputStreamFakerProvider( + resource, + config + ); + try { + provider.get().read(); + } + finally { + EasyMock.verify(resource); + } + } + + @Test + public void testTaskProvider() throws Exception + { + final Task task = new NoopTask("task_id", 0, 0, null, null, null); + final ObjectMapper mapper = EasyMock.createStrictMock(ObjectMapper.class); + EasyMock.expect(mapper.readValue(EasyMock.anyObject(File.class), EasyMock.eq(Task.class))).andReturn(task).once(); + EasyMock.replay(mapper); + final TaskProvider taskProvider = new TaskProvider(mapper, new ExecutorLifecycleConfig().setTaskFile( + temporaryFolder.newFile() + )); + Assert.assertEquals(task, taskProvider.get()); + } + + + @Test + public void testTaskProviderExceptional() throws Exception + { + final String msg = "test exception"; + final IOException ex = new IOException(msg); + expectedEx.expectCause(new BaseMatcher() + { + @Override + public void describeTo(Description description) + { + + } + + @Override + public boolean matches(Object o) + { + return ex == o; + } + }); + final ObjectMapper mapper = EasyMock.createStrictMock(ObjectMapper.class); + EasyMock.expect(mapper.readValue(EasyMock.anyObject(File.class), EasyMock.eq(Task.class))).andThrow(ex).once(); + EasyMock.replay(mapper); + final TaskProvider taskProvider = new TaskProvider(mapper, new ExecutorLifecycleConfig().setTaskFile( + temporaryFolder.newFile() + )); + taskProvider.get(); + } + + @After + public void tearDown() + { + // Clear interrupt flag + Thread.interrupted(); + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierLocalTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierLocalTest.java new file mode 100644 index 000000000000..f7ba8618e4ff --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierLocalTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.util.Modules; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Self; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.initialization.Initialization; +import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogPusher; +import org.easymock.EasyMock; +import org.junit.Test; + +public class CliTierLocalTest +{ + + @Test + public void testGetModules() throws Exception + { + final CliTierLocal cliTierLocal = new CliTierLocal(); + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of(Modules.override(cliTierLocal.getModules()).with(new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null) + ); + binder.bind(TaskLogPusher.class).toInstance(EasyMock.createNiceMock(TaskLogPusher.class)); + binder.bind(TaskConfig.class).toInstance(EasyMock.createNiceMock(TaskConfig.class)); + } + })) + ); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/ExecutorLifecycleProviderTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/ExecutorLifecycleProviderTest.java new file mode 100644 index 000000000000..801bd29c8d0b --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/ExecutorLifecycleProviderTest.java @@ -0,0 +1,229 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.SettableFuture; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.actions.TaskActionClientFactory; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.task.AbstractTask; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TierLocalTaskRunner; +import io.druid.indexing.worker.executor.ExecutorLifecycle; +import io.druid.jackson.DefaultObjectMapper; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.concurrent.CountDownLatch; + +@RunWith(Parameterized.class) +public class ExecutorLifecycleProviderTest +{ + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return Iterables.transform( + Arrays.asList(TaskStatus.Status.values()), + new Function() + { + @Nullable + @Override + public Object[] apply(TaskStatus.Status status) + { + return new Object[]{status}; + } + } + ); + } + + private static final String TASK_ID = "taskId"; + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + private final File taskFile = new File(TierLocalTaskRunner.TASK_FILE_NAME); + private final File logFile = new File(TierLocalTaskRunner.LOG_FILE_NAME); + private final File portFile = new File(TierLocalTaskRunner.PORT_FILE_NAME); + private final File statusFile = new File(TierLocalTaskRunner.STATUS_FILE_NAME); + + private final TaskStatus.Status status; + + public ExecutorLifecycleProviderTest(TaskStatus.Status status) throws IOException + { + this.status = status; + } + + @Before + public void setUp() + { + clearFiles(); + } + + @After + public void cleanUp() + { + clearFiles(); + } + + private void clearFiles() + { + taskFile.delete(); + logFile.delete(); + portFile.delete(); + statusFile.delete(); + } + + private void doTestTask(TaskStatus.Status retval) throws Exception + { + final TaskActionClientFactory taskActionClientFactory = EasyMock.createNiceMock(TaskActionClientFactory.class); + + final TaskRunner taskRunner = EasyMock.createNiceMock(TaskRunner.class); + final SettableFuture future = SettableFuture.create(); + EasyMock.expect(taskRunner.run(EasyMock.anyObject(Task.class))).andReturn(future).once(); + EasyMock.replay(taskRunner); + + final CountDownLatch neverEnd = new CountDownLatch(1); + final ParentMonitorInputStreamFaker parentStream = new ParentMonitorInputStreamFaker() + { + @Override + public int read() throws IOException + { + try { + neverEnd.await(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return -1; + } + }; + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + jsonMapper.registerSubtypes(TestTask.class); + final TaskConfig taskConfig = new TaskConfig( + temporaryFolder.newFolder().getAbsolutePath(), + temporaryFolder.newFolder().getAbsolutePath(), + temporaryFolder.newFolder().getAbsolutePath(), + null, + null, + false, + Period.seconds(1), + Period.seconds(1) + ); + final File taskLockParent = taskConfig.getTaskLockFile(TASK_ID).getParentFile(); + Assert.assertTrue(taskLockParent.isDirectory() || taskLockParent.mkdirs()); + final ExecutorLifecycleProvider provider = new ExecutorLifecycleProvider( + taskActionClientFactory, + taskRunner, + taskConfig, + parentStream, + jsonMapper + ); + + Assert.assertFalse(taskFile.exists()); + final TestTask task = new TestTask(retval, TASK_ID, "dataSource"); + jsonMapper.writeValue(taskFile, task); + final ExecutorLifecycle lifecycle = provider.get(); + Assert.assertTrue(statusFile.exists()); + lifecycle.start(); + future.set(TaskStatus.fromCode(task.getId(), retval)); + while (!statusFile.exists()) { + // NOOP + } + lifecycle.stop(); + final TaskStatus status = jsonMapper.readValue(statusFile, TaskStatus.class); + Assert.assertEquals(task.getId(), status.getId()); + Assert.assertEquals(retval, status.getStatusCode()); + } + + @Test(timeout = 1_000L) + public void testGet() throws Exception + { + doTestTask(status); + } + + @JsonTypeName(TestTask.TYPE) + public static class TestTask extends AbstractTask + { + static final String TYPE = "testSuccessTask"; + private final TaskStatus.Status status; + + @JsonCreator + public TestTask( + @JsonProperty("status") TaskStatus.Status status, + @JsonProperty("id") String id, + @JsonProperty("dataSource") @NotNull String dataSource + ) + { + super( + AbstractTask.makeId(id, TYPE, Preconditions.checkNotNull(dataSource), Interval.parse("2015/2016")), + dataSource, + ImmutableMap.of() + ); + this.status = status; + } + + @JsonProperty("status") + public TaskStatus.Status getStatus() + { + return status; + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return true; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + return TaskStatus.fromCode(getId(), status); + } + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/ForkAnnouncerTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/ForkAnnouncerTest.java new file mode 100644 index 000000000000..c5e2ff2aff0d --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/ForkAnnouncerTest.java @@ -0,0 +1,77 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.curator.announcement.Announcer; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.overlord.config.TierForkZkConfig; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.DruidNode; +import io.druid.server.initialization.ZkPathsConfig; +import org.easymock.EasyMock; +import org.junit.Test; + +public class ForkAnnouncerTest +{ + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + private static final DruidNode SELF = new DruidNode("myService", "myHost", 999); + private static final String TASK_ID = "test_task_id"; + + @Test + public void testAnnounceTask() throws Exception + { + final Announcer announcer = EasyMock.createStrictMock(Announcer.class); + final TierForkZkConfig zkConfig = new TierForkZkConfig(new ZkPathsConfig(), null, null); + final String path = zkConfig.getTierTaskIDPath(TASK_ID); + announcer.announce(EasyMock.eq(path), EasyMock.aryEq(MAPPER.writeValueAsBytes(SELF))); + EasyMock.expectLastCall().once(); + EasyMock.replay(announcer); + final ForkAnnouncer forkAnnouncer = new ForkAnnouncer( + announcer, + SELF, + MAPPER, + zkConfig, + new NoopTask(TASK_ID, 0, 0, "YES", null, null) + ); + forkAnnouncer.announceTask(); + EasyMock.verify(announcer); + } + + @Test + public void testUnannounceTask() throws Exception + { + final Announcer announcer = EasyMock.createStrictMock(Announcer.class); + final TierForkZkConfig zkConfig = new TierForkZkConfig(new ZkPathsConfig(), null, null); + final String path = zkConfig.getTierTaskIDPath(TASK_ID); + announcer.unannounce(EasyMock.eq(path)); + EasyMock.expectLastCall().once(); + EasyMock.replay(announcer); + final ForkAnnouncer forkAnnouncer = new ForkAnnouncer( + announcer, + SELF, + MAPPER, + zkConfig, + new NoopTask(TASK_ID, 0, 0, "YES", null, null) + ); + forkAnnouncer.unannounceTask(); + EasyMock.verify(announcer); + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunnerTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunnerTest.java new file mode 100644 index 000000000000..e5301f5232ad --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunnerTest.java @@ -0,0 +1,1044 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.base.Throwables; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.io.ByteSource; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.util.Modules; +import com.metamx.common.ISE; +import com.metamx.common.StringUtils; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceEventBuilder; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.response.ClientResponse; +import com.metamx.http.client.response.HttpResponseHandler; +import com.metamx.http.client.response.InputStreamResponseHandler; +import io.druid.cli.CliOverlord; +import io.druid.concurrent.Execs; +import io.druid.guice.GuiceInjectors; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.resources.TierRunningCheckResource; +import io.druid.initialization.Initialization; +import io.druid.server.DruidNode; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.jboss.netty.handler.codec.http.DefaultHttpResponse; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.jboss.netty.handler.codec.http.HttpVersion; +import org.joda.time.Duration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import javax.annotation.Nullable; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.MalformedURLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class AbstractTierRemoteTaskRunnerTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + private final HttpClient client = EasyMock.createStrictMock(HttpClient.class); + private final TierTaskDiscovery tierTaskDiscovery = EasyMock.createStrictMock(TierTaskDiscovery.class); + private final TaskStorage taskStorage = EasyMock.createStrictMock(TaskStorage.class); + private final Task task = NoopTask.create(); + private final Capture taskCapture = Capture.newInstance(); + private final DruidNode workerNode = new DruidNode("some service", "localhost", 0); + + private final ScheduledExecutorService executorService = Execs.scheduledSingleThreaded("AbstractTierTestService--%s"); + private final AbstractTierRemoteTaskRunner abstractTierRemoteTaskRunner = new AbstractTierRemoteTaskRunner( + tierTaskDiscovery, + client, + taskStorage, + new ScheduledExecutorFactory() + { + @Override + public ScheduledExecutorService create(int corePoolSize, String nameFormat) + { + return executorService; + } + } + ) + { + + @Override + protected void launch(SettableFuture future, Task task) + { + taskCapture.setValue(task); + } + }; + + @Before + public void generalSetup() + { + EasyMock.replay(client); + EasyMock.replay(tierTaskDiscovery); + EasyMock.replay(taskStorage); + noKnown(); + } + + private void useDiscoverySingleNode() + { + EasyMock.reset(tierTaskDiscovery); + EasyMock.expect(tierTaskDiscovery.getNodeForTask(EasyMock.eq(task.getId()))) + .andReturn(Optional.of(workerNode)).once(); + EasyMock.replay(tierTaskDiscovery); + } + + private void useDiscoveryMap() + { + EasyMock.reset(tierTaskDiscovery); + EasyMock.expect(tierTaskDiscovery.getTasks()) + .andReturn( + ImmutableMap.of( + task.getId(), + workerNode + ) + ).anyTimes(); + EasyMock.replay(tierTaskDiscovery); + } + + private void useGeneralClient() + { + final SettableFuture future = SettableFuture.create(); + future.set(true); + EasyMock.reset(client); + EasyMock.expect(client.go(EasyMock.anyObject(Request.class), EasyMock.anyObject(HttpResponseHandler.class))) + .andReturn(future) + .once(); + EasyMock.replay(client); + } + + private void noRunning() + { + Assert.assertTrue(abstractTierRemoteTaskRunner.getRunningTasks().isEmpty()); + } + + private void noPending() + { + Assert.assertTrue(abstractTierRemoteTaskRunner.getPendingTasks().isEmpty()); + } + + private void noKnown() + { + Assert.assertTrue(abstractTierRemoteTaskRunner.getKnownTasks().isEmpty()); + } + + private void onlyRunning() + { + noPending(); + testOnlyTasks(abstractTierRemoteTaskRunner.getKnownTasks(), ImmutableSet.of(task.getId())); + testOnlyTasks(abstractTierRemoteTaskRunner.getRunningTasks(), ImmutableSet.of(task.getId())); + } + + private void onlyPending() + { + noRunning(); + testOnlyTasks(abstractTierRemoteTaskRunner.getPendingTasks(), ImmutableSet.of(task.getId())); + testOnlyTasks(abstractTierRemoteTaskRunner.getKnownTasks(), ImmutableSet.of(task.getId())); + } + + private void onlyKnown() + { + noPending(); + noRunning(); + testOnlyTasks(abstractTierRemoteTaskRunner.getKnownTasks(), ImmutableSet.of(task.getId())); + } + + @Test + public void testSimpleRun() + { + final ListenableFuture future = submit(); + onlyPending(); + } + + @Test + public void testMultiStart() + { + expectedException.expect(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o instanceof ISE && "Already started".equals(((ISE) o).getMessage()); + } + + @Override + public void describeTo(Description description) + { + + } + }); + + useDiscoveryMap(); + useGeneralClient(); + abstractTierRemoteTaskRunner.start(); + onlyRunning(); + EasyMock.verify(tierTaskDiscovery); + abstractTierRemoteTaskRunner.start(); + } + + @Test + public void testSimpleStatusUpdatesWithOneRetry() throws InterruptedException, ExecutionException, TimeoutException + { + for (TaskStatus.Status status : TaskStatus.Status.values()) { + final String statuString = status.toString(); + EasyMock.reset(taskStorage); + + final ListenableFuture future = submit(); + + onlyPending(); + + final Capture taskIdCapture = Capture.newInstance(); + EasyMock.expect(taskStorage.getStatus(EasyMock.capture(taskIdCapture))).andReturn( + Optional.absent() + ).once().andReturn( + Optional.of( + TaskStatus.fromCode( + task.getId(), + status + ) + ) + ).once(); + EasyMock.replay(taskStorage); + + Assert.assertTrue( + statuString, + abstractTierRemoteTaskRunner.reportStatus(TaskStatus.fromCode(task.getId(), status)) + ); + + if (StatefulTaskRunnerWorkItem.State.of(status).isTerminal()) { + Assert.assertTrue(statuString, taskIdCapture.hasCaptured()); + Assert.assertEquals(statuString, task.getId(), taskIdCapture.getValue()); + EasyMock.verify(taskStorage); + + Assert.assertEquals(statuString, status, future.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + + final ListenableFuture future2 = abstractTierRemoteTaskRunner.run(task); + Assert.assertEquals(statuString, future, future2); + Assert.assertEquals(statuString, status, future2.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + } + abstractTierRemoteTaskRunner.stop(); // To clear the IDs + } + } + + @Test + public void testStatusUpdateOnUnknown() throws InterruptedException, ExecutionException, TimeoutException + { + for (TaskStatus.Status status : TaskStatus.Status.values()) { + EasyMock.reset(taskStorage); + + final Capture taskIdCapture = Capture.newInstance(); + EasyMock.expect(taskStorage.getStatus(EasyMock.capture(taskIdCapture))).andReturn( + Optional.of( + TaskStatus.fromCode( + task.getId(), + status + ) + ) + ).once(); + EasyMock.replay(taskStorage); + + noKnown(); + + final TaskStatus taskStatus = TaskStatus.fromCode(task.getId(), status); + Assert.assertTrue(abstractTierRemoteTaskRunner.reportStatus(taskStatus)); + if (!taskStatus.isComplete()) { + continue; + } + final ListenableFuture future = abstractTierRemoteTaskRunner.run(task); + + Assert.assertEquals(status, future.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + + Assert.assertTrue(taskIdCapture.hasCaptured()); + Assert.assertEquals(task.getId(), taskIdCapture.getValue()); + EasyMock.verify(taskStorage); + + final ListenableFuture future2 = abstractTierRemoteTaskRunner.run(task); + Assert.assertEquals(future, future2); + Assert.assertEquals(status, future2.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + abstractTierRemoteTaskRunner.stop(); // To clear the IDs + } + } + + + @Test + public void testStatusUpdateOnTerminalMismatch() throws InterruptedException, ExecutionException, TimeoutException + { + TaskStatus.Status status = TaskStatus.Status.FAILED; + EasyMock.reset(taskStorage); + + final Capture taskIdCapture = Capture.newInstance(); + EasyMock.expect(taskStorage.getStatus(EasyMock.capture(taskIdCapture))).andReturn( + Optional.of( + TaskStatus.fromCode( + task.getId(), + TaskStatus.Status.SUCCESS + ) + ) + ).once(); + EasyMock.replay(taskStorage); + + noKnown(); + + final TaskStatus taskStatus = TaskStatus.fromCode(task.getId(), status); + final ServiceEmitter serviceEmitter = EasyMock.createStrictMock(ServiceEmitter.class); + serviceEmitter.emit(EasyMock.anyObject(ServiceEventBuilder.class)); + EasyMock.expectLastCall().once(); + EasyMock.replay(serviceEmitter); + EmittingLogger.registerEmitter(serviceEmitter); + Assert.assertTrue(abstractTierRemoteTaskRunner.reportStatus(taskStatus)); + EasyMock.verify(serviceEmitter); + } + + @Test + public void testStatusUpdateErrorThenSuccess() throws InterruptedException, ExecutionException, TimeoutException + { + TaskStatus.Status status = TaskStatus.Status.SUCCESS; + EasyMock.reset(taskStorage); + + final RuntimeException testException = new RuntimeException("test exception"); + final Capture taskIdCapture = Capture.newInstance(); + EasyMock.expect(taskStorage.getStatus(EasyMock.capture(taskIdCapture))) + .andThrow(testException) + .once() + .andReturn( + Optional.of( + TaskStatus.fromCode( + task.getId(), + TaskStatus.Status.SUCCESS + ) + ) + ).once(); + EasyMock.replay(taskStorage); + + noKnown(); + + final TaskStatus taskStatus = TaskStatus.fromCode(task.getId(), status); + try { + abstractTierRemoteTaskRunner.reportStatus(taskStatus); + Assert.fail("Should not get here"); + } + catch (RuntimeException e) { + Assert.assertEquals("Exceptions did not match", testException, e); + } + onlyKnown(); + + Assert.assertTrue(abstractTierRemoteTaskRunner.reportStatus(taskStatus)); + final ListenableFuture future = abstractTierRemoteTaskRunner.run(task); + + Assert.assertEquals(status, future.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + + Assert.assertTrue(taskIdCapture.hasCaptured()); + Assert.assertEquals(task.getId(), taskIdCapture.getValue()); + EasyMock.verify(taskStorage); + + final ListenableFuture future2 = abstractTierRemoteTaskRunner.run(task); + Assert.assertEquals(future, future2); + Assert.assertEquals(status, future2.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + abstractTierRemoteTaskRunner.stop(); // To clear the IDs + } + + @Test + public void testStatusUpdateAfterRunning() throws InterruptedException, ExecutionException, TimeoutException + { + TaskStatus.Status status = TaskStatus.Status.SUCCESS; + EasyMock.reset(taskStorage); + + final Capture taskIdCapture = Capture.newInstance(); + EasyMock.expect(taskStorage.getStatus(EasyMock.capture(taskIdCapture))).andReturn( + Optional.of( + TaskStatus.fromCode( + task.getId(), + TaskStatus.Status.RUNNING + ) + ) + ).once().andReturn( + Optional.of( + TaskStatus.fromCode( + task.getId(), + TaskStatus.Status.SUCCESS + ) + ) + ).once(); + EasyMock.replay(taskStorage); + + noKnown(); + + final TaskStatus taskStatus = TaskStatus.fromCode(task.getId(), status); + Assert.assertFalse(abstractTierRemoteTaskRunner.reportStatus(taskStatus)); + Assert.assertTrue(abstractTierRemoteTaskRunner.reportStatus(taskStatus)); + final ListenableFuture future = abstractTierRemoteTaskRunner.run(task); + + Assert.assertEquals(status, future.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + + Assert.assertTrue(taskIdCapture.hasCaptured()); + Assert.assertEquals(task.getId(), taskIdCapture.getValue()); + EasyMock.verify(taskStorage); + + final ListenableFuture future2 = abstractTierRemoteTaskRunner.run(task); + Assert.assertEquals(future, future2); + Assert.assertEquals(status, future2.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + abstractTierRemoteTaskRunner.stop(); // To clear the IDs + } + + + @Test + public void testStatusUpdateMismatch() throws InterruptedException, ExecutionException, TimeoutException + { + final ListenableFuture future = submit(); + + Assert.assertTrue(((SettableFuture) future).set(TaskStatus.success(task.getId()))); + + testOnlyTasks(abstractTierRemoteTaskRunner.getKnownTasks(), ImmutableSet.of(task.getId())); + Assert.assertTrue(abstractTierRemoteTaskRunner.reportStatus(TaskStatus.failure(task.getId()))); + + Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + + final ListenableFuture future2 = abstractTierRemoteTaskRunner.run(task); + Assert.assertEquals(future, future2); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get(10, TimeUnit.MILLISECONDS).getStatusCode()); + } + + @Test + public void testSimpleStreamLog() throws IOException + { + useDiscoverySingleNode(); + final ListenableFuture future = submit(); + + final String testLog = "this is a log test"; + + final Optional byteSourceOptional = abstractTierRemoteTaskRunner.streamTaskLog(task.getId(), 0); + Assert.assertTrue(byteSourceOptional.isPresent()); + + // Request is lazy + final Capture requestCapture = Capture.newInstance(); + final Capture handlerCapture = Capture.newInstance(); + final SettableFuture clientFuture = SettableFuture.create(); + final ByteArrayInputStream bais = new ByteArrayInputStream(StringUtils.toUtf8(testLog)); + Assert.assertTrue(clientFuture.set(bais)); + EasyMock.resetToStrict(client); + EasyMock.expect(client.go( + EasyMock.capture(requestCapture), + EasyMock.capture(handlerCapture), + EasyMock.eq(Duration.parse("PT120s")) + )).andReturn( + clientFuture + ).once(); + EasyMock.replay(client); + Assert.assertEquals(testLog, byteSourceOptional.get().asCharSource(Charsets.UTF_8).read()); + Assert.assertTrue(requestCapture.hasCaptured()); + Assert.assertTrue(handlerCapture.hasCaptured()); + EasyMock.verify(client); + } + + @Test + public void testStreamLogInterrupted() throws Exception + { + final InterruptedException ex = new InterruptedException("test exception"); + expectedException.expectCause(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + useDiscoverySingleNode(); + final ListenableFuture future = submit(); + + final String testLog = "this is a log test"; + + final Optional byteSourceOptional = abstractTierRemoteTaskRunner.streamTaskLog(task.getId(), 0); + Assert.assertTrue(byteSourceOptional.isPresent()); + + // Request is lazy + final Capture requestCapture = Capture.newInstance(); + final Capture handlerCapture = Capture.newInstance(); + final ListenableFuture clientFuture = EasyMock.createStrictMock(ListenableFuture.class); + EasyMock.expect(clientFuture.get()).andThrow(ex).once(); + EasyMock.replay(clientFuture); + EasyMock.resetToStrict(client); + EasyMock.expect(client.go( + EasyMock.capture(requestCapture), + EasyMock.capture(handlerCapture), + EasyMock.eq(Duration.parse("PT120s")) + )).andReturn(clientFuture).once(); + EasyMock.replay(client); + Assert.assertEquals(testLog, byteSourceOptional.get().asCharSource(Charsets.UTF_8).read()); + Assert.assertTrue(requestCapture.hasCaptured()); + Assert.assertTrue(handlerCapture.hasCaptured()); + EasyMock.verify(client); + EasyMock.verify(clientFuture); + } + + + @Test + public void testStreamLogExecutionIOException() throws Exception + { + final IOException ex = new IOException("test exception"); + expectedException.expect(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + useDiscoverySingleNode(); + final ListenableFuture future = submit(); + + final String testLog = "this is a log test"; + + final Optional byteSourceOptional = abstractTierRemoteTaskRunner.streamTaskLog(task.getId(), 0); + Assert.assertTrue(byteSourceOptional.isPresent()); + + // Request is lazy + final Capture requestCapture = Capture.newInstance(); + final Capture handlerCapture = Capture.newInstance(); + final SettableFuture clientFuture = SettableFuture.create(); + clientFuture.setException(ex); + EasyMock.resetToStrict(client); + EasyMock.expect(client.go( + EasyMock.capture(requestCapture), + EasyMock.capture(handlerCapture), + EasyMock.eq(Duration.parse("PT120s")) + )).andReturn(clientFuture).once(); + EasyMock.replay(client); + Assert.assertEquals(testLog, byteSourceOptional.get().asCharSource(Charsets.UTF_8).read()); + Assert.assertTrue(requestCapture.hasCaptured()); + Assert.assertTrue(handlerCapture.hasCaptured()); + EasyMock.verify(client); + } + + + @Test + public void testStreamLogExecutionException() throws Exception + { + final Exception ex = new Exception("test exception"); + expectedException.expectCause(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + useDiscoverySingleNode(); + final ListenableFuture future = submit(); + + final String testLog = "this is a log test"; + + final Optional byteSourceOptional = abstractTierRemoteTaskRunner.streamTaskLog(task.getId(), 0); + Assert.assertTrue(byteSourceOptional.isPresent()); + + // Request is lazy + final Capture requestCapture = Capture.newInstance(); + final Capture handlerCapture = Capture.newInstance(); + final SettableFuture clientFuture = SettableFuture.create(); + clientFuture.setException(ex); + EasyMock.resetToStrict(client); + EasyMock.expect(client.go( + EasyMock.capture(requestCapture), + EasyMock.capture(handlerCapture), + EasyMock.eq(Duration.parse("PT120s")) + )).andReturn(clientFuture).once(); + EasyMock.replay(client); + Assert.assertEquals(testLog, byteSourceOptional.get().asCharSource(Charsets.UTF_8).read()); + Assert.assertTrue(requestCapture.hasCaptured()); + Assert.assertTrue(handlerCapture.hasCaptured()); + EasyMock.verify(client); + } + + + @Test + public void testStreamLogExecutionRuntimeException() throws Exception + { + final RuntimeException ex = new RuntimeException("test exception"); + expectedException.expect(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + useDiscoverySingleNode(); + final ListenableFuture future = submit(); + + final String testLog = "this is a log test"; + + final Optional byteSourceOptional = abstractTierRemoteTaskRunner.streamTaskLog(task.getId(), 0); + Assert.assertTrue(byteSourceOptional.isPresent()); + + // Request is lazy + final Capture requestCapture = Capture.newInstance(); + final Capture handlerCapture = Capture.newInstance(); + final SettableFuture clientFuture = SettableFuture.create(); + clientFuture.setException(ex); + EasyMock.resetToStrict(client); + EasyMock.expect(client.go( + EasyMock.capture(requestCapture), + EasyMock.capture(handlerCapture), + EasyMock.eq(Duration.parse("PT120s")) + )).andReturn(clientFuture).once(); + EasyMock.replay(client); + Assert.assertEquals(testLog, byteSourceOptional.get().asCharSource(Charsets.UTF_8).read()); + Assert.assertTrue(requestCapture.hasCaptured()); + Assert.assertTrue(handlerCapture.hasCaptured()); + EasyMock.verify(client); + } + + + @Test + public void testStreamMissingLog() throws IOException + { + EasyMock.reset(tierTaskDiscovery); + EasyMock.expect(tierTaskDiscovery.getNodeForTask(EasyMock.anyString())) + .andReturn(Optional.absent()).once(); + EasyMock.replay(tierTaskDiscovery); + Assert.assertFalse(abstractTierRemoteTaskRunner.streamTaskLog("does not exist", 0).isPresent()); + EasyMock.verify(tierTaskDiscovery); + } + + @Test + public void testMultipleUpdatesTaskIDs() throws InterruptedException, ExecutionException + { + final int numThreads = 10; + final int numRepeats = 10; + final ListeningExecutorService service = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + "listeningFork-%s" + ) + ); + final List> futureList = Lists.newArrayListWithExpectedSize(numThreads); + final CountDownLatch ready = new CountDownLatch(numThreads); + final CountDownLatch start = new CountDownLatch(1); + final CountDownLatch done = new CountDownLatch(numThreads); + + useDiscoveryMap(); + + for (int i = 0; i < numThreads; ++i) { + futureList.add( + service.submit( + new Runnable() + { + @Override + public void run() + { + try { + ready.countDown(); + start.await(); + for (int j = 0; j < numRepeats; ++j) { + abstractTierRemoteTaskRunner.refreshTaskIds(); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + finally { + done.countDown(); + } + } + } + ) + ); + } + try { + ready.await(1, TimeUnit.SECONDS); + start.countDown(); + done.await(10, TimeUnit.SECONDS); + } + finally { + service.shutdownNow(); + } + // Shake out any errors + Futures.allAsList(futureList).get(); + EasyMock.verify(tierTaskDiscovery); + + testOnlyTasks(abstractTierRemoteTaskRunner.getKnownTasks(), ImmutableSet.of(task.getId())); + } + + @Test + public void testSimpleRefreshTaskStatus() throws ExecutionException, InterruptedException + { + Assert.assertTrue(abstractTierRemoteTaskRunner.getKnownTasks().isEmpty()); + + useDiscoveryMap(); + + abstractTierRemoteTaskRunner.refreshTaskIds(); + testOnlyTasks(abstractTierRemoteTaskRunner.getKnownTasks(), ImmutableSet.of(task.getId())); + + final SettableFuture future = SettableFuture.create(); + future.set(true); + + final Capture requestCapture = Capture.newInstance(); + final Capture> responseHandlerCapture = Capture.newInstance(); + EasyMock.reset(client); + EasyMock.expect(client.go(EasyMock.capture(requestCapture), EasyMock.capture(responseHandlerCapture))) + .andReturn(future) + .once(); + EasyMock.replay(client); + abstractTierRemoteTaskRunner.refreshTaskStatus().get(); + + testOnlyTasks(abstractTierRemoteTaskRunner.getKnownTasks(), ImmutableSet.of(task.getId())); + + Assert.assertTrue(requestCapture.hasCaptured()); + Assert.assertTrue(responseHandlerCapture.hasCaptured()); + final Request request = requestCapture.getValue(); + Assert.assertEquals( + String.format("http://%s:%d%s", workerNode.getHost(), workerNode.getPort(), TierRunningCheckResource.PATH), + request.getUrl().toString() + ); + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + + Assert.assertTrue(responseHandlerCapture.hasCaptured()); + final HttpResponseHandler responseHandler = responseHandlerCapture.getValue(); + ClientResponse clientResponse = responseHandler.handleResponse( + new DefaultHttpResponse( + HttpVersion.HTTP_1_1, + HttpResponseStatus.OK + ) + ); + Assert.assertTrue(clientResponse.isFinished()); + clientResponse = responseHandler.handleResponse( + new DefaultHttpResponse( + HttpVersion.HTTP_1_1, + HttpResponseStatus.NOT_FOUND + ) + ); + Assert.assertTrue(clientResponse.isFinished()); + + responseHandler.done(ClientResponse.finished(true)); + onlyRunning(); + + responseHandler.done(ClientResponse.finished(false)); + onlyKnown(); + } + + + @Test + public void testRefreshTaskStatusMalformedURL() throws ExecutionException, InterruptedException + { + expectedException.expectCause(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o instanceof MalformedURLException; + } + + @Override + public void describeTo(Description description) + { + description.appendText("MalformedURLException"); + } + }); + + Assert.assertTrue(abstractTierRemoteTaskRunner.getKnownTasks().isEmpty()); + + useDiscoveryMap(); + + EasyMock.reset(tierTaskDiscovery); + EasyMock.expect(tierTaskDiscovery.getTasks()) + .andReturn( + ImmutableMap.of( + task.getId(), + new DruidNode("service", "localhost", -2) + ) + ).anyTimes(); + EasyMock.replay(tierTaskDiscovery); + + abstractTierRemoteTaskRunner.refreshTaskIds(); + testOnlyTasks(abstractTierRemoteTaskRunner.getKnownTasks(), ImmutableSet.of(task.getId())); + + abstractTierRemoteTaskRunner.refreshTaskStatus().get(); + } + + @Test + public void testRefreshTaskIdLostTask() throws ExecutionException, InterruptedException + { + noKnown(); + + useDiscoveryMap(); + + abstractTierRemoteTaskRunner.refreshTaskIds(); + + onlyRunning(); + + EasyMock.verify(tierTaskDiscovery); + EasyMock.reset(tierTaskDiscovery); + EasyMock.expect(tierTaskDiscovery.getTasks()).andReturn(ImmutableMap.of()).anyTimes(); + EasyMock.replay(tierTaskDiscovery); + + abstractTierRemoteTaskRunner.refreshTaskIds(); + + onlyKnown(); + + Assert.assertEquals( + StatefulTaskRunnerWorkItem.State.UNKNOWN, + ((StatefulTaskRunnerWorkItem) Iterables.getOnlyElement(abstractTierRemoteTaskRunner.getKnownTasks())).state + .get() + ); + } + + @Test + public void testFutureUpdatesRunState() throws ExecutionException, InterruptedException + { + final String taskId = task.getId(); + + final ListenableFuture future = submit(); + + onlyPending(); + + useDiscoveryMap(); + + abstractTierRemoteTaskRunner.refreshTaskIds(); + onlyRunning(); + + useGeneralClient(); + + abstractTierRemoteTaskRunner.refreshTaskStatus().get(); + + onlyRunning(); + + Assert.assertTrue(((SettableFuture) future).set(TaskStatus.success(taskId))); + + onlyKnown(); + } + + @Test + public void testSubmitHighlander() throws InterruptedException, ExecutionException, TimeoutException + { + final Task task = NoopTask.create(); + final int numThreads = 10; + final ListeningExecutorService service = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + "testSubmitter-%s" + ) + ); + final List>> futures = new ArrayList<>(); + final CountDownLatch ready = new CountDownLatch(numThreads); + final CountDownLatch go = new CountDownLatch(1); + + for (int i = 0; i < numThreads; ++i) { + futures.add( + service.submit( + new Callable>() + { + @Override + public ListenableFuture call() throws Exception + { + ready.countDown(); + try { + go.await(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + return abstractTierRemoteTaskRunner.run(task); + } + } + ) + ); + } + ready.await(1, TimeUnit.SECONDS); + go.countDown(); + final List> futureList = Futures.allAsList(futures).get(10, TimeUnit.SECONDS); + final ListenableFuture future = abstractTierRemoteTaskRunner.run(task); + for (ListenableFuture other : futureList) { + Assert.assertEquals(future, other); + } + } + + @Test + public void testUnknownShutdown() throws ExecutionException, InterruptedException + { + EasyMock.reset(tierTaskDiscovery); + EasyMock.expect(tierTaskDiscovery.getTasks()).andReturn(ImmutableMap.of()).once(); + EasyMock.replay(tierTaskDiscovery); + abstractTierRemoteTaskRunner.shutdown(task.getId()); + final ListenableFuture future = abstractTierRemoteTaskRunner.run(task); + Assert.assertTrue(future.get().isFailure()); + } + + @Test + public void testStartStop() + { + useDiscoveryMap(); + useGeneralClient(); + abstractTierRemoteTaskRunner.start(); + onlyRunning(); + EasyMock.verify(tierTaskDiscovery); + + abstractTierRemoteTaskRunner.stop(); + noKnown(); + + EasyMock.reset(tierTaskDiscovery); + EasyMock.expect(tierTaskDiscovery.getTasks()).andReturn(ImmutableMap.of()).once(); + EasyMock.replay(tierTaskDiscovery); + abstractTierRemoteTaskRunner.start(); + noKnown(); + } + + @Test + public void testInjection() + { + final String propName = "druid.indexer.runner.type"; + final String prior = System.getProperty(propName); + try { + System.setProperty(propName, "routing"); + final CliOverlord overlord = new CliOverlord(); + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of(Modules.override(overlord.getModules()).with(new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(ScheduledExecutorService.class).toInstance(executorService); + } + })) + ); + final TaskRunnerFactory factory = injector.getInstance(TaskRunnerFactory.class); + Assert.assertTrue(factory instanceof TierRoutingTaskRunnerFactory); + Assert.assertNotNull(factory.build()); + } + finally { + if (prior == null) { + System.clearProperty(propName); + } else { + System.setProperty(propName, prior); + } + } + } + + @After + public void generalVerify() + { + try { + EasyMock.verify(client); + EasyMock.verify(tierTaskDiscovery); + EasyMock.verify(taskStorage); + abstractTierRemoteTaskRunner.stop(); + noKnown(); + } + finally { + // Clear any potential interrupts + Thread.interrupted(); + } + } + + + private SettableFuture submit() + { + final ListenableFuture future = abstractTierRemoteTaskRunner.run(task); + Assert.assertEquals(SettableFuture.class, future.getClass()); + Assert.assertTrue(taskCapture.hasCaptured()); + Assert.assertEquals(task, taskCapture.getValue()); + return (SettableFuture) future; + } + + private static void testOnlyTasks(final Collection workItems, final Set taskIds) + { + final Set workIds = ImmutableSet.copyOf( + Collections2.transform( + workItems, new Function() + { + @Nullable + @Override + public String apply(TaskRunnerWorkItem taskRunnerWorkItem) + { + return taskRunnerWorkItem.getTaskId(); + } + } + ) + ); + Assert.assertEquals("exactly expected tasks not correct", taskIds, workIds); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/BusyTask.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/BusyTask.java new file mode 100644 index 000000000000..ac0754fdc517 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/BusyTask.java @@ -0,0 +1,109 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.logger.Logger; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.AbstractTask; + +import java.io.File; +import java.nio.file.Path; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +@JsonTypeName("busyTask") +public class BusyTask extends AbstractTask +{ + private static final Logger log = new Logger(BusyTask.class); + private final String lockFile; + private final long sleep; + + public BusyTask( + @JsonProperty("id") String id, + @JsonProperty("lockFile") String lockFile, + @JsonProperty("sleep") long sleep + ) + { + super( + id == null ? "testTask-" + UUID.randomUUID().toString() : id, + "noDataSource", + ImmutableMap.of() + ); + this.lockFile = lockFile; + this.sleep = sleep; + } + + @JsonProperty("lockFile") + public String getLockFile() + { + return lockFile; + } + + @JsonProperty("sleep") + public long getSleep() + { + return sleep; + } + + @Override + public String getType() + { + return "busyTask"; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return true; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + log.info("Deleting file at [%s]", getLockFile()); + File file = new File(getLockFile()); + if (!file.createNewFile()) { + log.error("Error deleting file at [%s]", file); + } + final Path path = file.toPath(); + while (file.exists()) { + try (WatchService service = path.getFileSystem().newWatchService()) { + path.getParent().register(service, StandardWatchEventKinds.ENTRY_DELETE); + if (file.exists()) { + WatchKey key = service.poll(sleep, TimeUnit.MILLISECONDS); + if (key == null) { + log.error("Ran out of time waiting for [%s]", path); + return TaskStatus.failure(getId()); + } + log.info("Delete event found for [%s]", path); + } + } + } + return TaskStatus.success(getId()); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/ForkingTaskRunnerTestModule.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/ForkingTaskRunnerTestModule.java new file mode 100644 index 000000000000..1e6f744f078c --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/ForkingTaskRunnerTestModule.java @@ -0,0 +1,92 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Key; +import io.druid.guice.LazySingleton; +import io.druid.guice.PolyBind; +import io.druid.initialization.DruidModule; +import io.druid.metadata.MetadataStorageConnector; +import io.druid.metadata.MetadataStorageConnectorConfig; +import io.druid.metadata.MetadataStorageTablesConfig; +import io.druid.metadata.SQLMetadataConnector; +import io.druid.metadata.storage.derby.DerbyConnector; +import org.apache.commons.dbcp2.BasicDataSource; +import org.apache.derby.jdbc.EmbeddedDriver; +import org.skife.jdbi.v2.DBI; + +import java.util.List; + +public class ForkingTaskRunnerTestModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + final SimpleModule module = new SimpleModule("ForkingTaskRunnerTestModule"); + module.registerSubtypes(BusyTask.class); + return ImmutableList.of(module); + } + + public static final String DB_TYPE = "test_derby_db"; + + @Override + public void configure(Binder binder) + { + PolyBind.optionBinder(binder, Key.get(MetadataStorageConnector.class)) + .addBinding(DB_TYPE) + .to(EmbeddedTestDerbyConnector.class) + .in(LazySingleton.class); + + PolyBind.optionBinder(binder, Key.get(SQLMetadataConnector.class)) + .addBinding(DB_TYPE) + .to(EmbeddedTestDerbyConnector.class) + .in(LazySingleton.class); + } +} + +class EmbeddedTestDerbyConnector extends DerbyConnector +{ + + @Inject + public EmbeddedTestDerbyConnector( + Supplier config, + Supplier dbTables + ) + { + super(config, dbTables, buildDBI(config)); + } + + private static DBI buildDBI(Supplier config) + { + final MetadataStorageConnectorConfig connectorConfig = config.get(); + final BasicDataSource basicDataSource = new BasicDataSource(); + basicDataSource.setDriver(new EmbeddedDriver()); + basicDataSource.setUrl(connectorConfig.getConnectURI() + ";create=true"); + basicDataSource.setValidationQuery("VALUES 1"); + basicDataSource.setTestOnBorrow(true); + return new DBI(basicDataSource); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/PortWriterTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/PortWriterTest.java new file mode 100644 index 000000000000..b549c9e1b572 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/PortWriterTest.java @@ -0,0 +1,171 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.base.Throwables; +import com.google.common.io.ByteStreams; +import com.metamx.common.ISE; +import com.metamx.common.StringUtils; +import io.druid.concurrent.Execs; +import io.druid.server.DruidNode; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.channels.OverlappingFileLockException; +import java.nio.file.StandardOpenOption; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +public class PortWriterTest +{ + private static final DruidNode me = new DruidNode("testService", "localhost", 8080); + private final File portFile = new File(TierLocalTaskRunner.PORT_FILE_NAME); + + @Before + public void setUp() + { + deletePortFile(); + } + + @After + public void tearDown() + { + deletePortFile(); + } + + private void deletePortFile() + { + Assert.assertTrue("cannot cleanup port file", (!portFile.exists() || portFile.delete()) || (!portFile.exists())); + } + + @Test + public void testPortWriting() throws IOException + { + Assert.assertFalse(portFile.exists()); + final PortWriter writer = new PortWriter(me); + writer.start(); + Assert.assertTrue(portFile.exists()); + final String portString; + try (final FileInputStream fis = new FileInputStream(portFile)) { + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + ByteStreams.copy(fis, baos); + portString = StringUtils.fromUtf8(baos.toByteArray()); + } + } + final int port = Integer.parseInt(portString); + Assert.assertEquals(me.getPort(), port); + writer.stop(); + Assert.assertFalse(portFile.exists()); + } + + @Test(expected = ISE.class) + public void testFailIfExists() throws IOException + { + Assert.assertFalse(portFile.exists()); + Assert.assertTrue(portFile.createNewFile()); + final PortWriter writer = new PortWriter(me); + writer.start(); + } + + // Unfortunately, the same JVM can't hold multiple instances of the lock + // This is just to verify expectations + @Test(expected = OverlappingFileLockException.class) + public void testLocking() throws IOException, InterruptedException, ExecutionException + { + final CountDownLatch readyLatch = new CountDownLatch(1); + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch doneLatch = new CountDownLatch(1); + final ExecutorService executorService = Execs.singleThreaded("testLocker"); + final Future future = executorService.submit( + new Runnable() + { + @Override + public void run() + { + try (FileChannel portFileChannel = FileChannel.open( + portFile.toPath(), + StandardOpenOption.CREATE_NEW, + StandardOpenOption.READ, + StandardOpenOption.WRITE + )) { + readyLatch.countDown(); + startLatch.await(); + final FileLock fileLock = portFileChannel.lock(); + try { + doneLatch.await(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + finally { + fileLock.release(); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + } + } + ); + readyLatch.await(); + try (FileChannel portFileChannel = FileChannel.open( + portFile.toPath(), + StandardOpenOption.READ, + StandardOpenOption.WRITE + )) { + final FileLock fileLock = portFileChannel.lock(); + try { + startLatch.countDown(); + Assert.assertFalse("Task did not lock!", doneLatch.await(100, TimeUnit.MILLISECONDS)); + if (future.isDone()) { + future.get(); + } + } + catch (ExecutionException e) { + Throwables.propagateIfInstanceOf(e.getCause(), OverlappingFileLockException.class); + throw e; + } + finally { + fileLock.release(); + } + // Eventually some future test may be able to reach here. + Assert.assertTrue("Unlock did not work!", doneLatch.await(100, TimeUnit.MILLISECONDS)); + } + finally { + executorService.shutdownNow(); + } + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierLocalTaskRunnerFactoryTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierLocalTaskRunnerFactoryTest.java new file mode 100644 index 000000000000..fc9010f65ecd --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierLocalTaskRunnerFactoryTest.java @@ -0,0 +1,87 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.http.client.HttpClient; +import io.druid.curator.discovery.ServiceAnnouncer; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; +import io.druid.indexing.worker.config.WorkerConfig; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogPusher; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Properties; + +public class TierLocalTaskRunnerFactoryTest +{ + + final TierLocalTaskRunnerConfig config = EasyMock.createStrictMock(TierLocalTaskRunnerConfig.class); + final TaskConfig taskConfig = EasyMock.createStrictMock(TaskConfig.class); + final WorkerConfig workerConfig = EasyMock.createStrictMock(WorkerConfig.class); + final Properties props = new Properties(); + final TaskLogPusher taskLogPusher = EasyMock.createStrictMock(TaskLogPusher.class); + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final DruidNode node = new DruidNode("some:service", "localhost", -1); + final HttpClient httpClient = EasyMock.createStrictMock(HttpClient.class); + final ServiceAnnouncer serviceAnnouncer = EasyMock.createStrictMock(ServiceAnnouncer.class); + + @Before + public void setUp() + { + EasyMock.replay(config, taskConfig, workerConfig, taskLogPusher, httpClient, serviceAnnouncer); + } + + @Test + public void testBuild() throws Exception + { + EasyMock.reset(workerConfig, config); + EasyMock.expect(workerConfig.getCapacity()).andReturn(1).times(2); + EasyMock.expect(config.getStartPort()).andReturn(9999).once(); + EasyMock.expect(config.getHeartbeatLocalNetworkTimeout()).andReturn(9999L).once(); + EasyMock.expect(config.getDelayBetweenHeartbeatBatches()).andReturn(1L).once(); + EasyMock.replay(workerConfig, config); + final TierLocalTaskRunnerFactory factory = new TierLocalTaskRunnerFactory( + config, + taskConfig, + workerConfig, + props, + taskLogPusher, + jsonMapper, + node, + httpClient, + serviceAnnouncer + ); + final TaskRunner runner = factory.build(); + Assert.assertTrue(runner instanceof TierLocalTaskRunner); + } + + @After + public void tearDown() + { + EasyMock.verify(config, taskConfig, workerConfig, taskLogPusher, httpClient, serviceAnnouncer); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierLocalTaskRunnerTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierLocalTaskRunnerTest.java new file mode 100644 index 000000000000..c8a90c2eb13d --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierLocalTaskRunnerTest.java @@ -0,0 +1,572 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.common.base.Predicate; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.io.ByteSink; +import com.google.common.io.ByteSource; +import com.google.common.io.Files; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.metamx.common.ISE; +import com.metamx.common.StringUtils; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import io.druid.curator.discovery.NoopServiceAnnouncer; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.Self; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.config.FileTaskLogsConfig; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.tasklogs.FileTaskLogs; +import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; +import io.druid.indexing.worker.config.WorkerConfig; +import io.druid.initialization.Initialization; +import io.druid.metadata.TestDerbyConnector; +import io.druid.segment.CloserRule; +import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogPusher; +import org.easymock.EasyMock; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +public class TierLocalTaskRunnerTest +{ + private static final Logger log = new Logger(TierLocalTaskRunnerTest.class); + @Rule + final public CloserRule closerRule = new CloserRule(true); + @Rule + final public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + final public TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private String taskId; + private File taskBaseDir; + private File taskDir; + private TierLocalTaskRunner forkingTaskRunner; + private HttpClient httpClient; + private File watchFile; + private ObjectMapper mapper; + private final AtomicInteger pushTaskLogCalls = new AtomicInteger(0); + private volatile File logDir; + + + private TierLocalTaskRunner makeForkingTaskRunner(Integer timeout) throws IOException + { + final Properties properties = new Properties(); + properties.setProperty("druid.processing.numThreads", "1"); + properties.setProperty( + "druid.metadata.storage.connector.connectURI", + derbyConnectorRule.getConnector().getJdbcUri() + ); + properties.setProperty("druid.indexer.task", ""); + properties.setProperty("druid.metadata.storage.type", ForkingTaskRunnerTestModule.DB_TYPE); + final TaskConfig taskConfig = new TaskConfig( + temporaryFolder.newFolder().getAbsolutePath(), + taskBaseDir.getAbsolutePath(), + "/tmp/TIER_LOCAL_TEST", + null, + null, + false, + Period.seconds(1), + Period.seconds(1) + ); + properties.setProperty("druid.indexer.task.baseTaskDir", taskConfig.getBaseTaskDir().toString()); + properties.setProperty("druid.indexer.task.baseDir", taskConfig.getBaseDir()); + final TierLocalTaskRunnerConfig config = new TierLocalTaskRunnerConfig(); + if (timeout != null) { + config.setSoftShutdownTimeLimit(timeout); + } + return new TierLocalTaskRunner( + config, + taskConfig, + new WorkerConfig() + { + @Override + public int getCapacity() + { + return 1; + } + }, + properties, + new TaskLogPusher() + { + @Override + public void pushTaskLog(String taskid, File logFile) throws IOException + { + pushTaskLogCalls.incrementAndGet(); + Files.move(logFile, getLogFile(taskid, logDir)); + } + }, + mapper, + new DruidNode("test/service", "localhost", -1), + httpClient, + new NoopServiceAnnouncer() + ) + { + @Override + public void stop() + { + super.stop(); + // Since we don't kill JVM between unit test instances, we want to make sure futures are trashed. + exec.shutdownNow(); + } + }; + } + + private static File getLogFile(String taskId, File logBase) + { + return Paths.get(logBase.getAbsolutePath(), String.format("%s.log", taskId)).toFile(); + } + + @Before + public void setUp() throws IOException + { + logDir = temporaryFolder.newFolder(); + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), ImmutableList.of( + new com.google.inject.Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null) + ); + try { + binder.bind(TaskLogPusher.class) + .toInstance(new FileTaskLogs(new FileTaskLogsConfig(temporaryFolder.newFolder()))); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); + + binder.bind(TaskRunner.class).toInstance(EasyMock.createNiceMock(TaskRunner.class)); + binder.bind(TaskStorage.class).toInstance(EasyMock.createNiceMock(TaskStorage.class)); + binder.bind(TaskMaster.class).toInstance(EasyMock.createNiceMock(TaskMaster.class)); + } + } + ) + ); + mapper = injector.getBinding(Key.get(ObjectMapper.class, Json.class)).getProvider().get(); + watchFile = new File(temporaryFolder.newFolder(), "watchFile"); + taskId = "BusyTaskID-" + UUID.randomUUID().toString(); + taskBaseDir = temporaryFolder.newFolder(); + taskDir = new File(taskBaseDir, taskId); + httpClient = injector.getInstance(Key.get(HttpClient.class, Global.class)); + forkingTaskRunner = makeForkingTaskRunner(30_000); + closerRule.closeLater( + new Closeable() + { + @Override + public void close() throws IOException + { + forkingTaskRunner.stop(); + } + } + ); + } + + @After + public void tearDown() throws IOException + { + Thread.interrupted(); + } + + @Test(timeout = 600_000) + public void testForkingCanKill() throws IOException, InterruptedException, ExecutionException + { + ListenableFuture future = waitForTaskStart(600_000); + Assert.assertFalse(forkingTaskRunner.getRunningTasks().isEmpty()); + forkingTaskRunner.shutdown(taskId); + Assert.assertTrue(future.get().isFailure()); + waitForEmptyTaskList(1_000); + Assert.assertFalse(taskDir.exists()); + } + + @Test(timeout = 600_000) + public void testForking() throws IOException, InterruptedException, ExecutionException + { + final ListenableFuture future = waitForTaskStart(60_000); + Assert.assertTrue(watchFile.delete()); + Assert.assertTrue( + FunctionalIterable + .create(forkingTaskRunner.getRunningTasks()) + .filter(new Predicate() + { + @Override + public boolean apply(TaskRunnerWorkItem input) + { + return taskId.equals(input.getTaskId()); + } + }).iterator().hasNext()); + Assert.assertFalse(forkingTaskRunner.streamTaskLog("djkfhjkafhds", 1).isPresent()); + Optional logSource = forkingTaskRunner.streamTaskLog(taskId, 1); + Assert.assertTrue(logSource.isPresent()); + try (InputStream inputStream = logSource.get().openStream()) { + Assert.assertTrue(inputStream.available() > 0); + } + TaskStatus status = future.get(); + if (!status.isSuccess()) { + logIfAvailable(taskId, logDir); + Assert.fail(String.format("Task [%s] failed", status.getId())); + } + Assert.assertFalse(taskDir.exists()); + Assert.assertFalse(forkingTaskRunner.streamTaskLog(taskId, 1).isPresent()); + } + + + @Test(timeout = 60_000, expected = IOException.class) + public void testScrewedUpTaskDir() throws Throwable + { + Assert.assertTrue(taskDir.createNewFile()); + forkingTaskRunner.start(); + final ListenableFuture future = forkingTaskRunner.run( + new BusyTask( + taskId, + watchFile.getAbsolutePath(), + 100 + ) + ); + try { + future.get(); + } + catch (ExecutionException e) { + throw e.getCause().getCause(); + } + } + + @Test(timeout = 600_000) + public void testKillingForkedJobNewRunner() + throws IOException, InterruptedException, ExecutionException, TimeoutException + { + forkingTaskRunner = makeForkingTaskRunner(1_000); + forkingTaskRunner.start(); + ListenableFuture future = waitForTaskStart(60_000); + Assert.assertFalse(forkingTaskRunner.getRunningTasks().isEmpty()); + log.info("Shutting down task [%s]", taskId); + forkingTaskRunner.shutdown(taskId); + Assert.assertTrue(future.get(1, TimeUnit.MINUTES).isFailure()); + waitForEmptyTaskList(1_000); + Assert.assertFalse(taskDir.exists()); + } + + @Test(timeout = 600_000) + public void testStartingNewRunner() throws IOException, InterruptedException, ExecutionException + { + log.info("Starting task"); + waitForTaskStart(600_000); + Assert.assertTrue(taskDir.exists()); + Assert.assertFalse(forkingTaskRunner.getRunningTasks().isEmpty()); + log.info("Stopping runner"); + forkingTaskRunner.stop(); + Assert.assertTrue(taskDir.exists()); + log.info("Creating new runner"); + forkingTaskRunner = makeForkingTaskRunner(600_000); + log.info("Starting new runner"); + forkingTaskRunner.start(); + // Should pick up prior task + Assert.assertFalse(forkingTaskRunner.getRunningTasks().isEmpty()); + log.info("Testing submission of prior task"); + ListenableFuture future = forkingTaskRunner.run(new BusyTask(taskId, null, 60_000L)); + // Signal task to exit + Assert.assertTrue(watchFile.delete()); + log.info("Waiting for task to complete"); + Assert.assertTrue(future.get().isSuccess()); + // Wait for task to clean up itself + log.info("Waiting for task to cleanup"); + if (taskDir.exists()) { + try (WatchService watchService = taskDir.toPath().getFileSystem().newWatchService()) { + taskDir.toPath().getParent().register(watchService, StandardWatchEventKinds.ENTRY_DELETE); + while (taskDir.exists()) { + Assert.assertNotNull(watchService.poll(1, TimeUnit.MINUTES)); + } + } + } + Assert.assertFalse(taskDir.exists()); + } + + + @Test + public void testAttemptDirCleanup() throws IOException + { + Assert.assertTrue(taskDir.mkdir()); + forkingTaskRunner.start(); + Assert.assertTrue(forkingTaskRunner.getRunningTasks().isEmpty()); + Assert.assertFalse(taskDir.exists()); + } + + @Test + public void testBadAttemptDir() throws IOException + { + final File attemptDir = new File(taskDir, "Cannot parse this!"); + Assert.assertTrue(attemptDir.mkdirs()); + + final File portFile = new File(attemptDir, "task.port"); + writeStringToFile("12345", portFile); + + final File taskFile = new File(attemptDir, "task.json"); + writeStringToFile( + mapper.writeValueAsString( + new BusyTask( + taskId, + watchFile.getAbsolutePath(), + 100 + ) + ), taskFile + ); + + final File logFile = new File(attemptDir, "task.log"); + Assert.assertTrue(logFile.createNewFile()); + + final File statusFile = new File(attemptDir, "status.json"); + Assert.assertTrue(statusFile.createNewFile()); + + forkingTaskRunner.start(); + Assert.assertTrue(forkingTaskRunner.getRunningTasks().isEmpty()); + } + + + @Test + public void testBadPort() throws IOException + { + final File attemptDir = new File(taskDir, "0000"); + Assert.assertTrue(attemptDir.mkdirs()); + + final File portFile = new File(attemptDir, "task.port"); + writeStringToFile("bad string", portFile); + + final File taskFile = new File(attemptDir, "task.json"); + writeStringToFile( + mapper.writeValueAsString( + new BusyTask( + taskId, + watchFile.getAbsolutePath(), + 100 + ) + ), taskFile + ); + + final File logFile = new File(attemptDir, "task.log"); + Assert.assertTrue(logFile.createNewFile()); + + final File statusFile = new File(attemptDir, "status.json"); + Assert.assertTrue(statusFile.createNewFile()); + + forkingTaskRunner.start(); + waitForEmptyTaskList(1_000); + } + + @Test + public void testBadTask() throws IOException + { + final File attemptDir = new File(taskDir, "0000"); + Assert.assertTrue(attemptDir.mkdirs()); + + final File portFile = new File(attemptDir, "task.port"); + writeStringToFile("12345", portFile); + + final File taskFile = new File(attemptDir, "task.json"); + writeStringToFile( + mapper.writeValueAsString( + new BusyTask( + taskId, + watchFile.getAbsolutePath(), + 100 + ) + ), taskFile + ); + + try (FileOutputStream fos = new FileOutputStream(taskFile)) { + fos.write(new byte[]{1, 2, 3, 4, 5, 6}); + } + + final File logFile = new File(attemptDir, "task.log"); + Assert.assertTrue(logFile.createNewFile()); + + final File statusFile = new File(attemptDir, "status.json"); + Assert.assertTrue(statusFile.createNewFile()); + + forkingTaskRunner.start(); + waitForEmptyTaskList(1_000); + } + + + @Test(expected = ISE.class) + public void testMultiStart() + { + forkingTaskRunner.start(); + forkingTaskRunner.start(); + } + + @Test + public void testMultiStop() + { + forkingTaskRunner.start(); + forkingTaskRunner.stop(); + forkingTaskRunner.stop(); + } + + @Test + public void testTaskHadFinished() throws IOException + { + final File attemptDir = new File(taskDir, "0000"); + Assert.assertTrue(attemptDir.mkdirs()); + + final File taskFile = new File(attemptDir, "task.json"); + writeStringToFile( + mapper.writeValueAsString( + new BusyTask( + taskId, + watchFile.getAbsolutePath(), + 100 + ) + ), taskFile + ); + + final File logFile = new File(attemptDir, "task.log"); + Assert.assertTrue(logFile.createNewFile()); + + final File statusFile = new File(attemptDir, "status.json"); + writeStringToFile(mapper.writeValueAsString(TaskStatus.success(taskId)), statusFile); + + Assert.assertEquals(0, pushTaskLogCalls.get()); + forkingTaskRunner.start(); + waitForEmptyTaskList(1_000); + Assert.assertEquals(1, pushTaskLogCalls.get()); + } + + private void waitForEmptyTaskList(long timeout) + { + long start = System.currentTimeMillis(); + while (!forkingTaskRunner.getRunningTasks().isEmpty()) { + Assert.assertTrue(System.currentTimeMillis() - start < timeout); + } + } + + private void writeStringToFile(String string, final File file) throws IOException + { + new ByteSink() + { + @Override + public OutputStream openStream() throws IOException + { + return new FileOutputStream(file); + } + }.write( + StringUtils.toUtf8(string) + ); + Assert.assertTrue(file.exists()); + Assert.assertTrue(file.length() > 0); + } + + private ListenableFuture waitForTaskStart(long sleep) throws InterruptedException, IOException + { + if (!forkingTaskRunner.isStarted(true)) { + forkingTaskRunner.start(); + } + log.info("Waiting for [%s] to appear", watchFile); + final Path watchPath = watchFile.toPath().getParent(); + Assert.assertFalse(watchFile.exists()); + try (final WatchService watchService = watchPath.getFileSystem().newWatchService()) { + watchPath.register(watchService, StandardWatchEventKinds.ENTRY_CREATE, StandardWatchEventKinds.ENTRY_DELETE); + final ListenableFuture future = forkingTaskRunner.run( + new BusyTask( + taskId, + watchFile.getAbsolutePath(), + sleep + ) + ); + // Kinda racy on exact size, so we just have it here for very high level test + Assert.assertTrue(forkingTaskRunner.getPendingTasks().size() >= 0); + + Assert.assertFalse(forkingTaskRunner.getKnownTasks().isEmpty()); + Assert.assertTrue( + FunctionalIterable + .create(forkingTaskRunner.getKnownTasks()) + .filter(new Predicate() + { + @Override + public boolean apply(TaskRunnerWorkItem input) + { + return taskId.equals(input.getTaskId()); + } + }).iterator().hasNext()); + while (!watchFile.exists() && !future.isDone()) { + WatchKey key = watchService.poll(100, TimeUnit.MILLISECONDS); + if (key != null) { + log.info("Event [%s]", key); + } + } + if (!watchFile.exists()) { + logIfAvailable(taskId, logDir); + Assert.fail(String.format("Failed to launch task [%s]", taskId)); + } + return future; + } + } + + private static void logIfAvailable(String taskId, File logDir) throws IOException + { + File logFile = getLogFile(taskId, logDir); + if (!logFile.exists()) { + log.warn("Log file [%s] for task [%s] does not exist", logFile, taskId); + return; + } + ByteSource source = Files.asByteSource(logFile); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + source.copyTo(baos); + log.info("Task [%s] log [%s]", taskId, StringUtils.fromUtf8(baos.toByteArray())); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierRoutingTaskRunnerTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierRoutingTaskRunnerTest.java new file mode 100644 index 000000000000..59c44c113895 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierRoutingTaskRunnerTest.java @@ -0,0 +1,308 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.google.common.base.Optional; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.io.ByteSource; +import com.google.common.util.concurrent.SettableFuture; +import com.metamx.common.ISE; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import io.druid.concurrent.Execs; +import io.druid.indexing.common.TaskLocation; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.overlord.autoscaling.AutoScalingData; +import io.druid.indexing.overlord.autoscaling.ScalingStats; +import io.druid.indexing.overlord.autoscaling.TierRoutingManagementStrategy; +import io.druid.indexing.overlord.routing.TierRouteConfig; +import io.druid.tasklogs.TaskLogStreamer; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collection; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +public class TierRoutingTaskRunnerTest +{ + private static final String TASK_ID = "some task id"; + private final AtomicReference executorService = new AtomicReference<>(); + private final ScheduledExecutorFactory scheduledExecutorServiceFactory = new ScheduledExecutorFactory() + { + @Override + public ScheduledExecutorService create(int corePoolSize, final String nameFormat) + { + if (!executorService.compareAndSet(null, Execs.scheduledSingleThreaded(nameFormat))) { + throw new ISE("Already created"); + } + return executorService.get(); + } + }; + private final TierRouteConfig tierRouteConfig = new TierRouteConfig(); + + private TierRoutingTaskRunner tierRoutingTaskRunner = null; + + @Before + public void setUp() + { + tierRoutingTaskRunner = new TierRoutingTaskRunner( + Suppliers.ofInstance(tierRouteConfig), + scheduledExecutorServiceFactory + ); + } + + @Test + public void testRestore() throws Exception + { + Assert.assertTrue(tierRoutingTaskRunner.restore().isEmpty()); + } + + @Test + public void testDefaultRun() throws Exception + { + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + final SettableFuture future = SettableFuture.create(); + EasyMock.expect(taskRunner.run(EasyMock.anyObject(NoopTask.class))).andReturn(future).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + Assert.assertEquals(future, tierRoutingTaskRunner.run(new NoopTask( + TASK_ID, + 0, + 0, + null, + null, + null + ))); + EasyMock.verify(taskRunner); + } + + @Test + public void testShutdown() throws Exception + { + // Test no routes + tierRoutingTaskRunner.shutdown(TASK_ID); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + taskRunner.shutdown(EasyMock.same(TASK_ID)); + EasyMock.expectLastCall().once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + tierRoutingTaskRunner.shutdown(TASK_ID); + EasyMock.verify(taskRunner); + } + + + @Test + public void testShutdownExceptional() throws Exception + { + // Test no routes + tierRoutingTaskRunner.shutdown(TASK_ID); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + taskRunner.shutdown(EasyMock.same(TASK_ID)); + EasyMock.expectLastCall().andThrow(new RuntimeException("test exception")).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + tierRoutingTaskRunner.shutdown(TASK_ID); + EasyMock.verify(taskRunner); + } + + @Test + public void testGetRunningTasks() throws Exception + { + Assert.assertTrue(tierRoutingTaskRunner.getRunningTasks().isEmpty()); + final Collection tasks = ImmutableList.of(new TaskRunnerWorkItem(TASK_ID, SettableFuture.create()) + { + @Override + public TaskLocation getLocation() + { + return null; + } + }); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(tasks).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + Assert.assertEquals(tasks, tierRoutingTaskRunner.getRunningTasks()); + EasyMock.verify(taskRunner); + } + + @Test + public void testGetRunningTasksExceptional() throws Exception + { + Assert.assertTrue(tierRoutingTaskRunner.getRunningTasks().isEmpty()); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + EasyMock.expect(taskRunner.getRunningTasks()).andThrow(new RuntimeException("test exception")).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + Assert.assertTrue(tierRoutingTaskRunner.getRunningTasks().isEmpty()); + EasyMock.verify(taskRunner); + } + + @Test + public void testGetPendingTasks() throws Exception + { + Assert.assertTrue(tierRoutingTaskRunner.getPendingTasks().isEmpty()); + final Collection tasks = ImmutableList.of(new TaskRunnerWorkItem(TASK_ID, SettableFuture.create()) + { + @Override + public TaskLocation getLocation() + { + return null; + } + }); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + EasyMock.expect(taskRunner.getPendingTasks()).andReturn(tasks).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + Assert.assertEquals(tasks, tierRoutingTaskRunner.getPendingTasks()); + EasyMock.verify(taskRunner); + } + + @Test + public void testGetPendingTasksExceptional() throws Exception + { + Assert.assertTrue(tierRoutingTaskRunner.getPendingTasks().isEmpty()); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + EasyMock.expect(taskRunner.getPendingTasks()).andThrow(new RuntimeException("test exception")).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + Assert.assertTrue(tierRoutingTaskRunner.getPendingTasks().isEmpty()); + EasyMock.verify(taskRunner); + } + + @Test + public void testGetKnownTasks() throws Exception + { + Assert.assertTrue(tierRoutingTaskRunner.getKnownTasks().isEmpty()); + final Collection tasks = ImmutableList.of(new TaskRunnerWorkItem(TASK_ID, SettableFuture.create()) + { + @Override + public TaskLocation getLocation() + { + return null; + } + }); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + EasyMock.expect(taskRunner.getKnownTasks()).andReturn(tasks).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + Assert.assertEquals(tasks, tierRoutingTaskRunner.getKnownTasks()); + EasyMock.verify(taskRunner); + } + + @Test + public void testGetKnownTasksExceptional() throws Exception + { + Assert.assertTrue(tierRoutingTaskRunner.getKnownTasks().isEmpty()); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + EasyMock.expect(taskRunner.getKnownTasks()).andThrow(new RuntimeException("test exception")).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + Assert.assertTrue(tierRoutingTaskRunner.getKnownTasks().isEmpty()); + EasyMock.verify(taskRunner); + } + + @Test + public void testGetScalingStats() throws Exception + { + Assert.assertFalse(tierRoutingTaskRunner.getScalingStats().isPresent()); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + final ScalingStats scalingStats = new ScalingStats(1); + scalingStats.addProvisionEvent(new AutoScalingData(ImmutableList.of("node0"))); + Assert.assertEquals(1, scalingStats.toList().size()); + EasyMock.expect(taskRunner.getScalingStats()).andReturn(Optional.of(scalingStats)).once(); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + final Optional maybeStats = tierRoutingTaskRunner.getScalingStats(); + EasyMock.verify(taskRunner); + Assert.assertTrue(maybeStats.isPresent()); + Assert.assertEquals(scalingStats.toList(), maybeStats.get().toList()); + } + + @Test + public void testStartStop() throws Exception + { + Assert.assertFalse(executorService.get().isShutdown()); + tierRoutingTaskRunner.start(); + tierRoutingTaskRunner.stop(); + Assert.assertTrue(executorService.get().isShutdown()); + } + + @Test + public void testStreamTaskLog() throws Exception + { + Assert.assertFalse(tierRoutingTaskRunner.streamTaskLog(TASK_ID, 0).isPresent()); + final TaskRunnerStreamer taskRunner = EasyMock.createStrictMock(TaskRunnerStreamer.class); + final ByteSource bs = ByteSource.empty(); + EasyMock.expect(taskRunner.streamTaskLog(EasyMock.same(TASK_ID), EasyMock.anyLong())).andReturn(Optional.of(bs)); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + final Optional maybeByteSource = tierRoutingTaskRunner.streamTaskLog(TASK_ID, 0); + Assert.assertTrue(maybeByteSource.isPresent()); + Assert.assertEquals(bs, maybeByteSource.get()); + EasyMock.verify(taskRunner); + } + + @Test + public void testStreamTaskLogExceptional() throws Exception + { + Assert.assertFalse(tierRoutingTaskRunner.streamTaskLog(TASK_ID, 0).isPresent()); + final TaskRunnerStreamer taskRunner = EasyMock.createStrictMock(TaskRunnerStreamer.class); + EasyMock.expect(taskRunner.streamTaskLog(EasyMock.same(TASK_ID), EasyMock.anyLong())) + .andThrow(new RuntimeException("test exception")); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + final Optional maybeByteSource = tierRoutingTaskRunner.streamTaskLog(TASK_ID, 0); + Assert.assertFalse(maybeByteSource.isPresent()); + EasyMock.verify(taskRunner); + } + + @Test + public void testStreamTaskLogSkipNotStreamer() throws Exception + { + Assert.assertFalse(tierRoutingTaskRunner.streamTaskLog(TASK_ID, 0).isPresent()); + final TaskRunner taskRunner = EasyMock.createStrictMock(TaskRunner.class); + EasyMock.replay(taskRunner); + tierRoutingTaskRunner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, taskRunner); + final Optional maybeByteSource = tierRoutingTaskRunner.streamTaskLog(TASK_ID, 0); + Assert.assertFalse(maybeByteSource.isPresent()); + EasyMock.verify(taskRunner); + } + + @After + public void tearDown() throws InterruptedException + { + final ScheduledExecutorService service = executorService.get(); + if (service != null) { + service.shutdownNow(); + service.awaitTermination(10, TimeUnit.SECONDS); + } + } +} + +abstract class TaskRunnerStreamer implements TaskRunner, TaskLogStreamer +{ + // nothing +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierTaskDiscoveryTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierTaskDiscoveryTest.java new file mode 100644 index 000000000000..1d4c27038ab7 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierTaskDiscoveryTest.java @@ -0,0 +1,186 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.name.Names; +import com.metamx.common.StringUtils; +import io.druid.concurrent.Execs; +import io.druid.curator.announcement.Announcer; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.annotations.Json; +import io.druid.indexing.overlord.config.TierForkZkConfig; +import io.druid.initialization.Initialization; +import io.druid.segment.CloserRule; +import io.druid.server.DruidNode; +import io.druid.server.initialization.ZkPathsConfig; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.RetryOneTime; +import org.apache.curator.test.TestingServer; +import org.apache.zookeeper.CreateMode; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.ExecutorService; + +public class TierTaskDiscoveryTest +{ + @Rule + public CloserRule closerRule = new CloserRule(true); + private final TierForkZkConfig tierForkZkConfig = new TierForkZkConfig(new ZkPathsConfig(), null, null); + private TestingServer zkTestServer; + private TierTaskDiscovery tierTaskDiscovery; + private ObjectMapper jsonMapper; + + @Before + public void setUp() throws Exception + { + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to( + "testService" + ); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + binder.bind(TaskRunner.class).toInstance(EasyMock.createNiceMock(TaskRunner.class)); + binder.bind(TaskStorage.class).toInstance(EasyMock.createNiceMock(TaskStorage.class)); + binder.bind(TaskMaster.class).toInstance(EasyMock.createNiceMock(TaskMaster.class)); + } + } + ) + ); + jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + final PortFinder portFinder = new PortFinder(8000); + zkTestServer = new TestingServer(portFinder.findUnusedPort()); + zkTestServer.start(); + closerRule.closeLater( + new Closeable() + { + @Override + public void close() throws IOException + { + zkTestServer.stop(); + } + } + ); + closerRule.closeLater(zkTestServer); + final CuratorFramework framework = CuratorFrameworkFactory.newClient( + String.format( + "localhost:%d", + zkTestServer.getPort() + ), new RetryOneTime(10) + ); + framework.start(); + framework.inTransaction().create().forPath("/" + tierForkZkConfig.zkPathsConfig.getBase()).and().commit(); + tierTaskDiscovery = new TierTaskDiscovery( + tierForkZkConfig, + closerRule.closeLater(framework), + jsonMapper + ); + } + + @Test + public void testSimpleDiscoNoNode() + { + Assert.assertTrue(tierTaskDiscovery.getTaskIDs().isEmpty()); + Assert.assertTrue(tierTaskDiscovery.getTasks().isEmpty()); + Assert.assertFalse(tierTaskDiscovery.getNodeForTask("does not exist").isPresent()); + } + + + @Test + public void testSimpleDiscoEmptyNode() throws Exception + { + final CuratorFramework curatorFramework = CuratorFrameworkFactory.newClient( + String.format( + "localhost:%d", + zkTestServer.getPort() + ), new RetryOneTime(10) + ); + curatorFramework.start(); + closerRule.closeLater(curatorFramework); + curatorFramework.blockUntilConnected(); + + curatorFramework.create().withMode(CreateMode.EPHEMERAL).forPath(tierForkZkConfig.getTierTaskIDPath()); + + Assert.assertTrue(tierTaskDiscovery.getTaskIDs().isEmpty()); + Assert.assertTrue(tierTaskDiscovery.getTasks().isEmpty()); + Assert.assertFalse(tierTaskDiscovery.getNodeForTask("does not exist").isPresent()); + } + + @Test + public void testDisco() throws Exception + { + final ExecutorService service = Execs.singleThreaded("testAnnouncer"); + final CuratorFramework curatorFramework = CuratorFrameworkFactory.newClient( + String.format( + "localhost:%d", + zkTestServer.getPort() + ), new RetryOneTime(10) + ); + curatorFramework.start(); + closerRule.closeLater(curatorFramework); + curatorFramework.blockUntilConnected(); + + final String host = "somehost"; + final String taskId = "taskId"; + final String type = "sometier"; + final int port = 9898; + final DruidNode testNode = new DruidNode(type, host, port); + final byte[] bytes = StringUtils.toUtf8(jsonMapper.writeValueAsString(testNode)); + + final Announcer announcer = new Announcer(curatorFramework, service); + announcer.start(); + closerRule.closeLater( + new Closeable() + { + @Override + public void close() throws IOException + { + announcer.stop(); + } + } + ); + announcer.announce(tierForkZkConfig.getTierTaskIDPath(taskId), bytes); + + while (tierTaskDiscovery.getTasks().isEmpty()) { + Thread.sleep(100); + } + + Assert.assertEquals(ImmutableList.of(taskId), ImmutableList.copyOf(tierTaskDiscovery.getTaskIDs())); + Assert.assertEquals(testNode, tierTaskDiscovery.getTasks().get(taskId)); + Assert.assertEquals(testNode, tierTaskDiscovery.getNodeForTask(taskId).get()); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java new file mode 100644 index 000000000000..a2fe9338d359 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java @@ -0,0 +1,228 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.autoscaling; + +import com.google.common.base.Optional; +import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import io.druid.concurrent.Execs; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.routing.TierRouteConfig; +import io.druid.indexing.overlord.routing.TierTaskRunnerFactory; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import javax.validation.constraints.NotNull; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; + +public class TierRoutingManagementStrategyTest +{ + private static final String TIER = "test_tier"; + private final ScheduledExecutorService scheduledExecutorService = Execs.scheduledSingleThreaded("routing-test-%s"); + private final TaskRunner runner = EasyMock.createStrictMock(TaskRunner.class); + private final Object memoryBarrier = new Object(); + private final ConcurrentHashMap taskRunnerMap = new ConcurrentHashMap<>(); + private final TierRouteConfig tierRouteConfig = new TierRouteConfig() + { + @Override + public Set getTiers() + { + return ImmutableSet.of(TIER); + } + + @Override + public + @NotNull + TierTaskRunnerFactory getRouteFactory(@NotNull String tier) + { + if (TIER.equals(tier)) { + return new TierTaskRunnerFactory() + { + @Override + public TaskRunner build() + { + synchronized (memoryBarrier) { + return runner; + } + } + }; + } else { + throw new IAE("Expected `%s` found `%s`", TIER, tier); + } + } + }; + + private TierRoutingManagementStrategy strategy = new TierRoutingManagementStrategy( + taskRunnerMap, + Suppliers.ofInstance(tierRouteConfig), + scheduledExecutorService + ); + + @Test + public void testStartManagement() throws Exception + { + synchronized (memoryBarrier) { + runner.start(); + EasyMock.expectLastCall().once(); + EasyMock.replay(runner); + } + final ExecutorService service = Execs.singleThreaded("TestWatcher"); + try { + final Future future = service.submit(new Runnable() + { + @Override + public void run() + { + try { + strategy.waitForUpdate(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + } + }); + strategy.startManagement(null); + future.get(); + EasyMock.verify(runner); + } + finally { + service.shutdownNow(); + } + } + + @Test(expected = ISE.class) + public void testMultipleStartsFails() throws Exception + { + strategy.startManagement(null); + strategy.startManagement(null); + } + + @Test(expected = ISE.class) + public void testMultipleStartsFailsAfterStop() throws Exception + { + strategy.startManagement(null); + strategy.stopManagement(); + strategy.startManagement(null); + } + + + @Test + public void testStopManagement() throws Exception + { + testStartManagement(); + synchronized (memoryBarrier) { + EasyMock.reset(runner); + runner.stop(); + EasyMock.expectLastCall().once(); + EasyMock.replay(runner); + } + strategy.stopManagement(); + Assert.assertTrue(scheduledExecutorService.isShutdown()); + EasyMock.verify(runner); + } + + @Test + public void testStopManagementMultiple() throws Exception + { + testStartManagement(); + synchronized (memoryBarrier) { + EasyMock.reset(runner); + runner.stop(); + EasyMock.expectLastCall().once(); + EasyMock.replay(runner); + } + strategy.stopManagement(); + strategy.stopManagement(); + EasyMock.verify(runner); + } + + @Test + public void testGetEmptyStats() throws Exception + { + testStartManagement(); + synchronized (memoryBarrier) { + EasyMock.reset(runner); + EasyMock.expect(runner.getScalingStats()).andReturn(Optional.absent()).once(); + EasyMock.replay(runner); + } + Assert.assertNull(strategy.getStats()); + EasyMock.verify(runner); + } + + @Test + public void testSimpleOneStates() throws Exception + { + testStartManagement(); + final ScalingStats.ScalingEvent event = EasyMock.createStrictMock(ScalingStats.ScalingEvent.class); + synchronized (memoryBarrier) { + final ScalingStats stats = new ScalingStats(0); + stats.addAllEvents(ImmutableList.of(event)); + EasyMock.reset(runner); + EasyMock.expect(runner.getScalingStats()).andReturn(Optional.of(stats)).once(); + EasyMock.replay(runner); + } + final ScalingStats stats = strategy.getStats(); + EasyMock.verify(runner); + Assert.assertFalse(stats.toList().isEmpty()); + Assert.assertEquals(event, stats.toList().get(0)); + } + + @Test + public void testGetRunner() throws Exception + { + final TaskRunner defaultRunner = EasyMock.createStrictMock(TaskRunner.class); + taskRunnerMap.put(TierRoutingManagementStrategy.DEFAULT_ROUTE, defaultRunner); + testStartManagement(); + Assert.assertEquals(runner, strategy.getRunner( + new NoopTask("task_id", 0, 0, "YES", null, ImmutableMap.of( + TierRoutingManagementStrategy.ROUTING_TARGET_CONTEXT_KEY, + TIER + )) + )); + Assert.assertEquals(defaultRunner, strategy.getRunner( + new NoopTask("task_id", 0, 0, "YES", null, ImmutableMap.of()) + )); + + Assert.assertEquals(defaultRunner, strategy.getRunner( + new NoopTask("task_id", 0, 0, "YES", null, null) + )); + } + + @After + public void tearDown() + { + if (!scheduledExecutorService.isShutdown()) { + scheduledExecutorService.shutdownNow(); + } + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandMonitorTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandMonitorTest.java new file mode 100644 index 000000000000..efb0cbef3408 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandMonitorTest.java @@ -0,0 +1,143 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + +import com.metamx.common.ISE; +import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +public class DeadhandMonitorTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + private File deadhandFile; + private static final long timeout = 100L; + DeadhandMonitor monitor; + private final AtomicLong exitCalled = new AtomicLong(0L); + private final CountDownLatch exitCalledLatch = new CountDownLatch(1); + + @Before + public void setUp() throws IOException + { + deadhandFile = temporaryFolder.newFile(); + exitCalled.set(0L); + monitor = new DeadhandMonitor( + new DeadhandResource(), + new TierLocalTaskRunnerConfig() + { + @Override + public long getHeartbeatTimeLimit() + { + return timeout; + } + }, + deadhandFile + ) + { + @Override + void exit() + { + exitCalled.incrementAndGet(); + exitCalledLatch.countDown(); + } + }; + } + + @After + public void tearDown() + { + monitor.stop(); + } + + @Test + public void testStartStop() + { + Assert.assertEquals(0, exitCalled.get()); + monitor.start(); + Assert.assertEquals(0, exitCalled.get()); + monitor.stop(); + Assert.assertEquals(0, exitCalled.get()); + } + + @Test(expected = ISE.class) + public void testStartAfterStop() + { + Assert.assertEquals(0, exitCalled.get()); + monitor.start(); + Assert.assertEquals(0, exitCalled.get()); + monitor.stop(); + Assert.assertEquals(0, exitCalled.get()); + monitor.start(); + } + + @Test + public void testWackyStartStop() + { + Assert.assertEquals(0, exitCalled.get()); + monitor.start(); + monitor.start(); + monitor.start(); + monitor.start(); + monitor.start(); + monitor.start(); + monitor.start(); + Assert.assertEquals(0, exitCalled.get()); + monitor.stop(); + monitor.stop(); + monitor.stop(); + monitor.stop(); + monitor.stop(); + Assert.assertEquals(0, exitCalled.get()); + } + + @Test + public void testTimeout() throws InterruptedException + { + Assert.assertEquals(0, exitCalled.get()); + Thread.sleep(timeout * 2); + Assert.assertEquals(0, exitCalled.get()); + monitor.start(); + exitCalledLatch.await(timeout * 3, TimeUnit.MILLISECONDS); + Assert.assertNotEquals(0, exitCalled.get()); + } + + @Test + public void testDeadhandFile() throws InterruptedException + { + Assert.assertTrue(deadhandFile.delete()); + Assert.assertEquals(0, exitCalled.get()); + Thread.sleep(timeout * 2); + Assert.assertEquals(0, exitCalled.get()); + monitor.start(); + exitCalledLatch.await(timeout * 3, TimeUnit.MILLISECONDS); + Assert.assertNotEquals(0, exitCalled.get()); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandResourceTest.java new file mode 100644 index 000000000000..e242b87b0dc4 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandResourceTest.java @@ -0,0 +1,303 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + +import com.google.common.base.Throwables; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import io.druid.concurrent.Execs; +import io.druid.segment.CloserRule; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.core.Response; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class DeadhandResourceTest +{ + @Rule + public CloserRule closerRule = new CloserRule(true); + DeadhandResource deadhandResource; + + @Before + public void setUp() + { + deadhandResource = new DeadhandResource(); + } + + @Test(timeout = 10_000L) + public void testConcurrentDoHeartbeat() throws Exception + { + final HttpServletRequest request = EasyMock.createStrictMock(HttpServletRequest.class); + EasyMock.expect(request.getRemoteAddr()).andReturn("localhost").anyTimes(); // to make sure easymock threadsafety + EasyMock.replay(request); + + final int numThreads = 10; + final ListeningExecutorService service = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + "testThread-%s" + ) + ); + closerRule.closeLater(executorAsCloseable(service)); + final CountDownLatch readyLatch = new CountDownLatch(numThreads); + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch doneLatch = new CountDownLatch(numThreads); + final Collection> futures = new ArrayList<>(numThreads); + for (int i = 0; i < numThreads; ++i) { + futures.add( + service.submit( + new Runnable() + { + @Override + public void run() + { + readyLatch.countDown(); + try { + startLatch.await(); + Assert.assertEquals( + Response.Status.OK.getStatusCode(), + deadhandResource.doHeartbeat(request).getStatus() + ); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + finally { + doneLatch.countDown(); + } + } + } + ) + ); + } + readyLatch.await(1, TimeUnit.SECONDS); + startLatch.countDown(); + doneLatch.await(); + Assert.assertEquals(numThreads, deadhandResource.getHeartbeatCount()); + EasyMock.verify(request); + Assert.assertFalse(Futures.allAsList(futures).get().isEmpty()); + } + + @Test + public void testDoHeartbeat() throws Exception + { + HttpServletRequest request = EasyMock.createStrictMock(HttpServletRequest.class); + EasyMock.expect(request.getRemoteAddr()).andReturn("localhost").once(); + EasyMock.replay(request); + Assert.assertEquals(Response.Status.OK.getStatusCode(), deadhandResource.doHeartbeat(request).getStatus()); + EasyMock.verify(request); + } + + + @Test(expected = TimeoutException.class) + public void testWaitTimeout() throws Exception + { + deadhandResource.waitForHeartbeat(5); + } + + @Test + public void testWaitForHeartbeat() throws Exception + { + final ExecutorService service = Execs.singleThreaded("testThread-%s"); + closerRule.closeLater(executorAsCloseable(service)); + final Future future = service.submit( + new Runnable() + { + @Override + public void run() + { + try { + deadhandResource.waitForHeartbeat(1_000); + } + catch (InterruptedException | TimeoutException e) { + throw Throwables.propagate(e); + } + } + } + ); + while (!future.isDone()) { + testDoHeartbeat(); + Thread.sleep(1); + } + future.get(2_000, TimeUnit.MILLISECONDS); + } + + @Test + public void testConcurrentWaitForHeartbeat() throws Exception + { + final int numThreads = 10; + final ListeningExecutorService service = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + "testThread-%s" + ) + ); + closerRule.closeLater(executorAsCloseable(service)); + final CountDownLatch readyLatch = new CountDownLatch(numThreads); + final CountDownLatch startLatch = new CountDownLatch(1); + final Collection> futures = new ArrayList<>(numThreads); + for (int i = 0; i < numThreads; ++i) { + futures.add( + service.submit( + new Runnable() + { + @Override + public void run() + { + try { + readyLatch.countDown(); + startLatch.await(); + deadhandResource.waitForHeartbeat(10_000); + } + catch (InterruptedException | TimeoutException e) { + throw Throwables.propagate(e); + } + } + } + ) + ); + } + service.shutdown(); + readyLatch.await(1, TimeUnit.SECONDS); + startLatch.countDown(); + final ListenableFuture collectiveFuture = Futures.allAsList(futures); + while (!collectiveFuture.isDone()) { + testDoHeartbeat(); + Thread.sleep(1); + } + collectiveFuture.get(20, TimeUnit.SECONDS); + } + + + @Test(timeout = 10_000L) + public void testConcurrentDoHeartbeatAndWait() throws Exception + { + final HttpServletRequest request = EasyMock.createStrictMock(HttpServletRequest.class); + EasyMock.expect(request.getRemoteAddr()).andReturn("localhost").anyTimes(); // to make sure easymock threadsafety + EasyMock.replay(request); + + final int numThreads = 10; + final ListeningExecutorService beatingService = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + "testBeatingThread-%s" + ) + ); + final ListeningExecutorService listeningService = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + "testListeningThread-%s" + ) + ); + + closerRule.closeLater(executorAsCloseable(beatingService)); + closerRule.closeLater(executorAsCloseable(listeningService)); + + final CountDownLatch readyLatch = new CountDownLatch(numThreads * 2); + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch doneLatch = new CountDownLatch(numThreads * 2); + final Collection> futures = new ArrayList<>(numThreads * 2); + + for (int i = 0; i < numThreads; ++i) { + futures.add( + beatingService.submit( + new Runnable() + { + @Override + public void run() + { + readyLatch.countDown(); + try { + startLatch.await(); + Assert.assertEquals( + Response.Status.OK.getStatusCode(), + deadhandResource.doHeartbeat(request).getStatus() + ); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + finally { + doneLatch.countDown(); + } + } + } + ) + ); + futures.add( + listeningService.submit( + new Runnable() + { + @Override + public void run() + { + readyLatch.countDown(); + try { + startLatch.await(); + deadhandResource.waitForHeartbeat(5_000); + } + catch (InterruptedException | TimeoutException e) { + throw Throwables.propagate(e); + } + finally { + doneLatch.countDown(); + } + } + } + ) + ); + } + + readyLatch.await(1, TimeUnit.SECONDS); + startLatch.countDown(); + doneLatch.await(); + Assert.assertEquals(numThreads, deadhandResource.getHeartbeatCount()); + EasyMock.verify(request); + + Assert.assertFalse(Futures.allAsList(futures).get().isEmpty()); + } + + private static Closeable executorAsCloseable(final ExecutorService service) + { + return new Closeable() + { + @Override + public void close() throws IOException + { + service.shutdownNow(); + } + }; + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java new file mode 100644 index 000000000000..b3a4fdc20ba9 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java @@ -0,0 +1,153 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + +import com.google.common.io.ByteStreams; +import com.metamx.common.StringUtils; +import io.druid.indexing.overlord.TierLocalTaskRunner; +import io.druid.server.DruidNode; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.ws.rs.core.Response; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.InputStream; +import java.net.URL; + +public class TaskLogResourceTest +{ + private static final File logFile = new File(TierLocalTaskRunner.LOG_FILE_NAME); + + @Before + public void setUp() + { + deleteLogFile(); + } + + @After + public void tearDown() + { + deleteLogFile(); + } + + private void deleteLogFile() + { + Assert.assertTrue("cannot cleanup log file", (!logFile.exists() || logFile.delete()) || (!logFile.exists())); + } + + @Test + public void testGetNoLog() throws Exception + { + final TaskLogResource taskLogResource = new TaskLogResource(); + Assert.assertFalse(logFile.exists()); + final Response response = taskLogResource.getLog(0); + Assert.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), response.getStatus()); + } + + + @Test + public void testGetEmptyLog() throws Exception + { + final TaskLogResource taskLogResource = new TaskLogResource(); + Assert.assertFalse(logFile.exists()); + Assert.assertTrue(logFile.createNewFile()); + final Response response = taskLogResource.getLog(0); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + final Object entity = response.getEntity(); + Assert.assertTrue(entity instanceof InputStream); + try (InputStream fis = (InputStream) entity) { + Assert.assertEquals(0, fis.available()); + } + } + + @Test + public void testNegativeOffset() throws Exception + { + final TaskLogResource taskLogResource = new TaskLogResource(); + Assert.assertFalse(logFile.exists()); + Assert.assertTrue(logFile.createNewFile()); + final Response response = taskLogResource.getLog(-1); + Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); + } + + @Test + public void testLogWithOffset() throws Exception + { + final String string = "some test string for logging"; + final byte[] data = StringUtils.toUtf8(string); + try (final FileOutputStream fos = new FileOutputStream(logFile)) { + fos.write(data); + } + + final TaskLogResource taskLogResource = new TaskLogResource(); + Assert.assertTrue(logFile.exists()); + for (int i = 0; i < data.length; ++i) { + final Response response = taskLogResource.getLog(i); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + final Object entity = response.getEntity(); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(data.length - i); + try (InputStream fis = (InputStream) entity) { + ByteStreams.copy(fis, baos); + } + Assert.assertEquals(string.substring(i), StringUtils.fromUtf8(baos.toByteArray())); + } + } + + @Test + public void testLogPastEnd() throws Exception + { + final TaskLogResource taskLogResource = new TaskLogResource(); + Assert.assertFalse(logFile.exists()); + Assert.assertTrue(logFile.createNewFile()); + for (int i = 1; i < 100; ++i) { + final Response response = taskLogResource.getLog(i); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + final Object entity = response.getEntity(); + Assert.assertTrue(entity instanceof InputStream); + try (InputStream fis = (InputStream) entity) { + Assert.assertEquals(0, fis.available()); + } + } + } + + @Test + public void testBuildURL() throws Exception + { + final DruidNode testNode = new DruidNode("testService", "somehost", 1234); + for (int i = 0; i < 10000; ++i) { + final URL url = TaskLogResource.buildURL(testNode, i); + Assert.assertEquals("http", url.getProtocol()); + Assert.assertEquals(testNode.getHost(), url.getHost()); + Assert.assertEquals(testNode.getPort(), url.getPort()); + Assert.assertEquals(String.format("%s=%d", TaskLogResource.OFFSET_PARAM, i), url.getQuery()); + } + } + + @Test(expected = IllegalArgumentException.class) + public void testBuildURLBadOffset() throws Exception + { + final DruidNode testNode = new DruidNode("testService", "somehost", 1234); + TaskLogResource.buildURL(testNode, -1); + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TierRunningCheckResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TierRunningCheckResourceTest.java new file mode 100644 index 000000000000..480106ad97a6 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TierRunningCheckResourceTest.java @@ -0,0 +1,38 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.resources; + +import org.junit.Assert; +import org.junit.Test; + +import javax.ws.rs.core.Response; + +public class TierRunningCheckResourceTest +{ + + @Test + public void testGet() throws Exception + { + final TierRunningCheckResource tierRunningCheckResource = new TierRunningCheckResource(); + final Response response = tierRunningCheckResource.get(); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertNull(response.getEntity()); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java new file mode 100644 index 000000000000..d678e7fefbac --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java @@ -0,0 +1,147 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.SettableFuture; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskRunner; +import org.apache.curator.x.discovery.ServiceDiscovery; +import org.apache.curator.x.discovery.ServiceInstance; +import org.easymock.EasyMock; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.Duration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class DynamicUpstreamReporterTest +{ + private static final String DEFAULT_UPSTREAM = "upstream_service"; + final Object memoryBarrier = new Object(); + TaskStatusReporterModule.DynamicUpstreamReporter reporter = null; + final TaskMaster taskMaster = EasyMock.createStrictMock(TaskMaster.class); + final HttpClient httpClient = EasyMock.createStrictMock(HttpClient.class); + final ServiceDiscovery serviceDiscovery = EasyMock.createStrictMock(ServiceDiscovery.class); + final ServiceInstance service = EasyMock.createStrictMock(ServiceInstance.class); + final TaskRunnerReporter runner = EasyMock.createStrictMock(TaskRunnerReporter.class); + final TaskStatus status = TaskStatus.success("task_id"); + + final + @Before + public void setUp() throws Exception + { + EasyMock.expect(serviceDiscovery.queryForInstances(EasyMock.eq(DEFAULT_UPSTREAM))) + .andReturn(ImmutableList.of(service)) + .anyTimes(); + EasyMock.expect(runner.reportStatus(EasyMock.eq(status))).andReturn(true).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(runner)).anyTimes(); + EasyMock.replay(serviceDiscovery, taskMaster, httpClient, runner); + reporter = new TaskStatusReporterModule.DynamicUpstreamReporter( + taskMaster, + httpClient, + serviceDiscovery, + DEFAULT_UPSTREAM + ); + } + + @Test + public void testReportStatusSimple() throws Exception + { + Assert.assertTrue(reporter.reportStatus(status)); + } + + @Test + public void testReportStatusMissingRunner() throws Exception + { + synchronized (memoryBarrier) { + EasyMock.reset(taskMaster); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).once(); + EasyMock.replay(taskMaster); + } + testPost(); + } + + @Test + public void testReportStatusSillyRunner() throws Exception + { + synchronized (memoryBarrier) { + final TaskRunner notReporter = EasyMock.createStrictMock(TaskRunner.class); + EasyMock.reset(taskMaster); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(notReporter)).once(); + EasyMock.replay(taskMaster); + } + testPost(); + } + + @Test + public void testReportStatusNullManager() throws Exception + { + reporter = new TaskStatusReporterModule.DynamicUpstreamReporter( + null, + httpClient, + serviceDiscovery, + DEFAULT_UPSTREAM + ); + testPost(); + } + + + private void testPost() + { + synchronized (memoryBarrier) { + EasyMock.reset(service); + EasyMock.expect(service.getAddress()).andReturn("localhost").once(); + EasyMock.expect(service.getPort()).andReturn(0).once(); + EasyMock.replay(service); + + final SettableFuture future = SettableFuture.create(); + future.set(new StatusResponseHolder(HttpResponseStatus.ACCEPTED, new StringBuilder())); + + EasyMock.reset(httpClient); + EasyMock.expect(httpClient.go( + EasyMock.anyObject(Request.class), + EasyMock.anyObject(StatusResponseHandler.class), + EasyMock.eq(Duration.parse("PT60s")) + )).andReturn(future).once(); + EasyMock.replay(httpClient); + } + Assert.assertTrue(reporter.reportStatus(status)); + } + + @After + public void tearDown() + { + EasyMock.verify(serviceDiscovery, taskMaster, httpClient, runner); + } +} + +abstract class TaskRunnerReporter implements TaskRunner, TaskStatusReporter +{ + // Nothing +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java new file mode 100644 index 000000000000..be1f8fa848a6 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java @@ -0,0 +1,93 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.http.client.HttpClient; +import io.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerResourceManagementConfig; +import io.druid.indexing.overlord.autoscaling.ResourceManagementSchedulerConfig; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.initialization.IndexerZkConfig; +import org.apache.curator.framework.CuratorFramework; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +public class RemoteTaskRunnerTierFactoryTest +{ + final RemoteTaskRunnerConfig remoteTaskRunnerConfig = EasyMock.createStrictMock(RemoteTaskRunnerConfig.class); + final PendingTaskBasedWorkerResourceManagementConfig config = EasyMock.createStrictMock( + PendingTaskBasedWorkerResourceManagementConfig.class); + final Supplier workerConfigRef = Suppliers.ofInstance(EasyMock.createStrictMock( + WorkerBehaviorConfig.class)); + final CuratorFramework curator = EasyMock.createStrictMock(CuratorFramework.class); + final IndexerZkConfig zkPaths = EasyMock.createStrictMock(IndexerZkConfig.class); + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final HttpClient httpClient = EasyMock.createStrictMock(HttpClient.class); + final ScheduledExecutorFactory executorFactory = EasyMock.createStrictMock(ScheduledExecutorFactory.class); + final ResourceManagementSchedulerConfig resourceManagementSchedulerConfig = + EasyMock.createStrictMock(ResourceManagementSchedulerConfig.class); + final RemoteTaskRunnerTierFactory factory = new RemoteTaskRunnerTierFactory( + remoteTaskRunnerConfig, + config, + workerConfigRef, + curator, + zkPaths, + jsonMapper, + httpClient, + executorFactory, + resourceManagementSchedulerConfig + ); + + @Test + public void testBuild() throws Exception + { + + } + + @Test + public void testGetRemoteTaskRunnerConfig() throws Exception + { + Assert.assertEquals(remoteTaskRunnerConfig, factory.getRemoteTaskRunnerConfig()); + } + + @Test + public void testGetZkPaths() throws Exception + { + Assert.assertEquals(zkPaths, factory.getZkPaths()); + } + + @Test + public void testGetPendingConfig() throws Exception + { + Assert.assertEquals(config, factory.getPendingTaskBasedWorkerResourceManagementConfig()); + } + + @Test + public void testGetResourceManagementSchedulerConfig() throws Exception + { + Assert.assertEquals(remoteTaskRunnerConfig, factory.getRemoteTaskRunnerConfig()); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResourceTest.java new file mode 100644 index 000000000000..7ae266dc180e --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResourceTest.java @@ -0,0 +1,111 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import io.druid.indexing.common.TaskStatus; +import io.druid.server.DruidNode; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +import javax.ws.rs.core.Response; +import java.net.URL; + +public class TaskStatusPostToLeaderListenerResourceTest +{ + private static final TaskStatus status = TaskStatus.success("task_id"); + + @Test + public void testDoPostGood() throws Exception + { + final TaskStatusReporter reporter = EasyMock.createStrictMock(TaskStatusReporter.class); + EasyMock.expect(reporter.reportStatus(EasyMock.anyObject(TaskStatus.class))).andReturn(true).once(); + EasyMock.replay(reporter); + final TaskStatusPostToLeaderListenerResource resource = new TaskStatusPostToLeaderListenerResource(reporter); + final Response response = resource.doPost(status); + EasyMock.verify(reporter); + Assert.assertEquals(Response.Status.ACCEPTED.getStatusCode(), response.getStatus()); + } + + @Test + public void testDoPostBad() throws Exception + { + final TaskStatusReporter reporter = EasyMock.createStrictMock(TaskStatusReporter.class); + EasyMock.expect(reporter.reportStatus(EasyMock.anyObject(TaskStatus.class))).andReturn(false).once(); + EasyMock.replay(reporter); + final TaskStatusPostToLeaderListenerResource resource = new TaskStatusPostToLeaderListenerResource(reporter); + final Response response = resource.doPost(status); + EasyMock.verify(reporter); + Assert.assertEquals(Response.Status.SERVICE_UNAVAILABLE.getStatusCode(), response.getStatus()); + } + + @Test + public void testDoPostERROR() throws Exception + { + final RuntimeException ex = new RuntimeException("test exception"); + final TaskStatusReporter reporter = EasyMock.createStrictMock(TaskStatusReporter.class); + EasyMock.expect(reporter.reportStatus(EasyMock.anyObject(TaskStatus.class))).andThrow(ex).once(); + EasyMock.replay(reporter); + final TaskStatusPostToLeaderListenerResource resource = new TaskStatusPostToLeaderListenerResource(reporter); + final Response response = resource.doPost(status); + EasyMock.verify(reporter); + Assert.assertEquals(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), response.getStatus()); + } + + @Test + public void testMakeReportUrl() throws Exception + { + final String host = "localhost"; + final int port = 1; + final DruidNode node = new DruidNode("service_name", host, port); + final URL url = TaskStatusPostToLeaderListenerResource.makeReportUrl(node); + Assert.assertEquals(host, url.getHost()); + Assert.assertEquals(port, url.getPort()); + Assert.assertEquals("http", url.getProtocol()); + Assert.assertEquals(TaskStatusPostToLeaderListenerResource.PATH, url.getFile()); + } + + + @Test + public void testMakeReportUrlIPv6() throws Exception + { + final int port = 1; + final String host = "::1"; + final DruidNode node = new DruidNode("service_name", host, port); + final URL url = TaskStatusPostToLeaderListenerResource.makeReportUrl(node); + Assert.assertEquals("[::1]", url.getHost()); + Assert.assertEquals(port, url.getPort()); + Assert.assertEquals("http", url.getProtocol()); + Assert.assertEquals(TaskStatusPostToLeaderListenerResource.PATH, url.getFile()); + } + + @Test + public void testMakeReportUrlIPv6Bracketed() throws Exception + { + final int port = 1; + final String host = "[::1]"; + final DruidNode node = new DruidNode("service_name", host, port); + final URL url = TaskStatusPostToLeaderListenerResource.makeReportUrl(node); + Assert.assertEquals(host, url.getHost()); + Assert.assertEquals(port, url.getPort()); + Assert.assertEquals("http", url.getProtocol()); + Assert.assertEquals(TaskStatusPostToLeaderListenerResource.PATH, url.getFile()); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporterTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporterTest.java new file mode 100644 index 000000000000..c34d0c7c2a72 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporterTest.java @@ -0,0 +1,383 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import com.metamx.common.ISE; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskRunner; +import org.apache.curator.x.discovery.ServiceDiscovery; +import org.apache.curator.x.discovery.ServiceInstance; +import org.easymock.EasyMock; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.Duration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.net.MalformedURLException; +import java.util.concurrent.ExecutionException; + +public class TaskStatusPostToLeaderReporterTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + private static final String DEFAULT_UPSTREAM = "upstream_service"; + final Object memoryBarrier = new Object(); + TaskStatusPostToLeaderReporter reporter = null; + final TaskMaster taskMaster = EasyMock.createStrictMock(TaskMaster.class); + final HttpClient httpClient = EasyMock.createStrictMock(HttpClient.class); + final ServiceDiscovery serviceDiscovery = EasyMock.createStrictMock(ServiceDiscovery.class); + final ServiceInstance service = EasyMock.createStrictMock(ServiceInstance.class); + final TaskRunnerReporter runner = EasyMock.createStrictMock(TaskRunnerReporter.class); + final TaskStatus status = TaskStatus.success("task_id"); + + final + @Before + public void setUp() throws Exception + { + EasyMock.expect(serviceDiscovery.queryForInstances(EasyMock.eq(DEFAULT_UPSTREAM))) + .andReturn(ImmutableList.of(service)) + .once(); + EasyMock.expect(service.getAddress()).andReturn("localhost").once(); + EasyMock.expect(service.getPort()).andReturn(0).once(); + EasyMock.expect(runner.reportStatus(EasyMock.eq(status))).andReturn(true).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(runner)).anyTimes(); + EasyMock.replay(serviceDiscovery, taskMaster, httpClient, runner, service); + reporter = new TaskStatusPostToLeaderReporter( + httpClient, + serviceDiscovery, + DEFAULT_UPSTREAM + ); + } + + @Test + public void testReportStatus() throws Exception + { + synchronized (memoryBarrier) { + final SettableFuture future = SettableFuture.create(); + future.set(new StatusResponseHolder(HttpResponseStatus.ACCEPTED, new StringBuilder())); + EasyMock.reset(httpClient); + EasyMock.expect(httpClient.go( + EasyMock.anyObject(Request.class), + EasyMock.anyObject(StatusResponseHandler.class), + EasyMock.eq(Duration.parse("PT60s")) + )).andReturn(future).once(); + EasyMock.replay(httpClient); + } + Assert.assertTrue(reporter.reportStatus(status)); + } + + + @Test + public void testReportStatusFailed() throws Exception + { + synchronized (memoryBarrier) { + final SettableFuture future = SettableFuture.create(); + future.set(new StatusResponseHolder(HttpResponseStatus.SERVICE_UNAVAILABLE, new StringBuilder())); + EasyMock.reset(httpClient); + EasyMock.expect(httpClient.go( + EasyMock.anyObject(Request.class), + EasyMock.anyObject(StatusResponseHandler.class), + EasyMock.eq(Duration.parse("PT60s")) + )).andReturn(future).once(); + EasyMock.replay(httpClient); + } + Assert.assertFalse(reporter.reportStatus(status)); + } + + + @Test + public void testReportStatusException() throws Exception + { + synchronized (memoryBarrier) { + final SettableFuture future = SettableFuture.create(); + future.set(new StatusResponseHolder(HttpResponseStatus.BAD_GATEWAY, new StringBuilder())); + EasyMock.reset(httpClient); + EasyMock.expect(httpClient.go( + EasyMock.anyObject(Request.class), + EasyMock.anyObject(StatusResponseHandler.class), + EasyMock.eq(Duration.parse("PT60s")) + )).andReturn(future).once(); + EasyMock.replay(httpClient); + } + expectedException.expect(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + if (o instanceof ISE) { + final ISE ise = (ISE) o; + return ise.getMessage().startsWith("Unknown response ["); + } + return false; + } + + @Override + public void describeTo(Description description) + { + + } + }); + reporter.reportStatus(status); + } + + @Test + public void testReportStatusExecutionException() throws Exception + { + final RuntimeException ex = new RuntimeException("test exception"); + synchronized (memoryBarrier) { + final SettableFuture future = SettableFuture.create(); + future.setException(ex); + EasyMock.reset(httpClient); + EasyMock.expect(httpClient.go( + EasyMock.anyObject(Request.class), + EasyMock.anyObject(StatusResponseHandler.class), + EasyMock.eq(Duration.parse("PT60s")) + )).andReturn(future).once(); + EasyMock.replay(httpClient); + } + expectedException.expectCause(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return (o instanceof ExecutionException) && ((ExecutionException) o).getCause() == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + reporter.reportStatus(status); + } + + @Test + public void testReportStatusExecutionInterrupted() throws Exception + { + final InterruptedException ex = new InterruptedException("test exception"); + final ListenableFuture future = EasyMock.createStrictMock(ListenableFuture.class); + synchronized (memoryBarrier) { + EasyMock.expect(future.get()).andThrow(ex).once(); + EasyMock.replay(future); + EasyMock.reset(httpClient); + EasyMock.expect(httpClient.go( + EasyMock.anyObject(Request.class), + EasyMock.anyObject(StatusResponseHandler.class), + EasyMock.eq(Duration.parse("PT60s")) + )).andReturn(future).once(); + EasyMock.replay(httpClient); + } + expectedException.expectCause(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + try { + reporter.reportStatus(status); + } + finally { + EasyMock.verify(future); + } + } + + + @Test + public void testReportStatusMalformedURL() throws Exception + { + synchronized (memoryBarrier) { + EasyMock.reset(service); + EasyMock.expect(service.getAddress()).andReturn("localhost").once(); + EasyMock.expect(service.getPort()).andReturn(-2).once(); + EasyMock.replay(service); + } + expectedException.expectCause(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o instanceof MalformedURLException; + } + + @Override + public void describeTo(Description description) + { + + } + }); + reporter.reportStatus(status); + } + + @Test + public void testReportStatusNoOverlord() throws Exception + { + synchronized (memoryBarrier) { + EasyMock.reset(serviceDiscovery); + EasyMock.expect(serviceDiscovery.queryForInstances(EasyMock.eq(DEFAULT_UPSTREAM))) + .andReturn(ImmutableList.>of()) + .once(); + EasyMock.replay(serviceDiscovery); + EasyMock.reset(service); + EasyMock.replay(service); + } + expectedException.expect(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o instanceof ISE && ((ISE) o).getMessage() + .equals(String.format( + "No overlords found for service [%s]", + DEFAULT_UPSTREAM + )); + } + + @Override + public void describeTo(Description description) + { + + } + }); + reporter.reportStatus(status); + } + + + @Test + public void testReportStatusQueryInterrupted() throws Exception + { + final InterruptedException ex = new InterruptedException("test exception"); + synchronized (memoryBarrier) { + EasyMock.reset(serviceDiscovery); + EasyMock.expect(serviceDiscovery.queryForInstances(EasyMock.eq(DEFAULT_UPSTREAM))) + .andThrow(ex).once(); + EasyMock.replay(serviceDiscovery); + EasyMock.reset(service); + EasyMock.replay(service); + } + expectedException.expectCause(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + reporter.reportStatus(status); + } + + @Test + public void testReportStatusQueryRuntimeException() throws Exception + { + final RuntimeException ex = new RuntimeException("test exception"); + synchronized (memoryBarrier) { + EasyMock.reset(serviceDiscovery); + EasyMock.expect(serviceDiscovery.queryForInstances(EasyMock.eq(DEFAULT_UPSTREAM))) + .andThrow(ex).once(); + EasyMock.replay(serviceDiscovery); + EasyMock.reset(service); + EasyMock.replay(service); + } + expectedException.expect(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + reporter.reportStatus(status); + } + + + @Test + public void testReportStatusQueryException() throws Exception + { + final Exception ex = new Exception("test exception"); + synchronized (memoryBarrier) { + EasyMock.reset(serviceDiscovery); + EasyMock.expect(serviceDiscovery.queryForInstances(EasyMock.eq(DEFAULT_UPSTREAM))) + .andThrow(ex).once(); + EasyMock.replay(serviceDiscovery); + EasyMock.reset(service); + EasyMock.replay(service); + } + expectedException.expectCause(new BaseMatcher() + { + @Override + public boolean matches(Object o) + { + return o == ex; + } + + @Override + public void describeTo(Description description) + { + + } + }); + reporter.reportStatus(status); + } + + @After + public void tearDown() + { + // Clear interrupt + Thread.interrupted(); + synchronized (memoryBarrier) { + EasyMock.verify(serviceDiscovery, taskMaster, httpClient, runner, service); + } + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusReporterModuleTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusReporterModuleTest.java new file mode 100644 index 000000000000..0142b735fd33 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusReporterModuleTest.java @@ -0,0 +1,48 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +public class TaskStatusReporterModuleTest +{ + @Test + public void testUpstreamNameHolder() throws Exception + { + final String name = "foo"; + final TaskStatusReporterModule.UpstreamNameHolder upstreamNameHolder = new TaskStatusReporterModule.UpstreamNameHolder(); + upstreamNameHolder.upstreamServiceName = name; + Assert.assertEquals(name, upstreamNameHolder.getUpstreamServiceName()); + final ObjectMapper mapper = new DefaultObjectMapper(); + Assert.assertEquals( + name, + mapper.readValue( + mapper.writeValueAsString(upstreamNameHolder), + TaskStatusReporterModule.UpstreamNameHolder.class + ).getUpstreamServiceName() + ); + final TaskStatusReporterModule.UpstreamNameHolderProvider provider = new TaskStatusReporterModule.UpstreamNameHolderProvider( + upstreamNameHolder); + Assert.assertEquals(name, provider.get()); + } +} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TierRouteConfigTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TierRouteConfigTest.java new file mode 100644 index 000000000000..8e4a4dd983bb --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TierRouteConfigTest.java @@ -0,0 +1,90 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import io.druid.guice.GuiceInjectors; +import io.druid.initialization.Initialization; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class TierRouteConfigTest +{ + private static final String TIER = "tier"; + private static final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of(new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + } + }) + ); + private final TierRouteConfig routeConfig = new TierRouteConfig(); + private final TierTaskRunnerFactory factory = new UnknownRouteFactory(); + + @Before + public void setUp() + { + routeConfig.mapper = injector.getInstance(ObjectMapper.class); + routeConfig.tierMap = new HashMap<>(); + } + + @Test + public void testGetRouteFactory() throws Exception + { + populateConfig(TIER, factory); + Assert.assertTrue(routeConfig.getRouteFactory(TIER) instanceof UnknownRouteFactory); + } + + @Test + public void testGetTiers() throws Exception + { + Assert.assertEquals(ImmutableSet.of(), routeConfig.getTiers()); + populateConfig(TIER, factory); + Assert.assertEquals(ImmutableSet.of(TIER), routeConfig.getTiers()); + } + + private void populateConfig(final String tier, final TierTaskRunnerFactory factory) + { + routeConfig.tierMap.put( + tier, + routeConfig.mapper.>convertValue( + factory, + new TypeReference>() + { + } + ) + ); + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/UnknownRouteFactoryTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/UnknownRouteFactoryTest.java new file mode 100644 index 000000000000..ba733ff821e4 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/UnknownRouteFactoryTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.metamx.common.UOE; +import org.junit.Test; + +public class UnknownRouteFactoryTest +{ + + @Test(expected = UOE.class) + public void testBuild() throws Exception + { + new UnknownRouteFactory().build(); + } +} \ No newline at end of file diff --git a/extensions-core/tasktier/src/test/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/tasktier/src/test/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..173643e16f94 --- /dev/null +++ b/extensions-core/tasktier/src/test/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1,19 @@ +# +# Licensed to Metamarkets Group Inc. (Metamarkets) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. Metamarkets licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +io.druid.indexing.overlord.ForkingTaskRunnerTestModule diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java index aa9161638db6..04c44c650051 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java @@ -34,6 +34,7 @@ */ public class ForkingTaskRunnerFactory implements TaskRunnerFactory { + public static final String TYPE_NAME = "forking"; private final ForkingTaskRunnerConfig config; private final TaskConfig taskConfig; private final WorkerConfig workerConfig; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ScalingStats.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ScalingStats.java index b68482ccce90..94d12b72fc90 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ScalingStats.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ScalingStats.java @@ -26,6 +26,7 @@ import com.google.common.collect.Ordering; import org.joda.time.DateTime; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -91,6 +92,13 @@ public void addTerminateEvent(AutoScalingData data) } } + public void addAllEvents(Collection events) + { + synchronized (lock) { + recentEvents.addAll(events); + } + } + @JsonValue public List toList() { diff --git a/pom.xml b/pom.xml index ae9bb6e340fd..ea46a1be9ed2 100644 --- a/pom.xml +++ b/pom.xml @@ -93,6 +93,7 @@ extensions-core/postgresql-metadata-storage extensions-core/lookups-cached-global extensions-core/s3-extensions + extensions-core/tasktier extensions-contrib/azure-extensions extensions-contrib/cassandra-storage @@ -103,7 +104,7 @@ extensions-contrib/rabbitmq extensions-contrib/distinctcount extensions-contrib/parquet-extensions - extensions-contrib/statsd-emitter + extensions-contrib/statsd-emitter distribution @@ -111,6 +112,38 @@ + + + io.druid + druid-api + ${project.version} + + + io.druid + druid-services + ${project.version} + + + io.druid + druid-processing + ${project.version} + + + io.druid + druid-common + ${project.version} + + + io.druid + druid-server + ${project.version} + + + io.druid + druid-indexing-service + ${project.version} + + com.metamx @@ -122,6 +155,12 @@ http-client 1.0.4 + + + io.netty + netty + 3.9.5.Final + com.metamx java-util @@ -241,6 +280,10 @@ org.jboss.netty netty + + io.netty + netty + @@ -408,6 +451,11 @@ jsr305 2.0.1 + + org.slf4j + slf4j-api + ${slf4j.version} + org.apache.logging.log4j log4j-api diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 3aea375a774c..d0545cf7444b 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -108,7 +108,7 @@ public CliOverlord() } @Override - protected List getModules() + public List getModules() { return ImmutableList.of( new Module() From 35794a73c3a906b373f90ecfceb5a41c15708d72 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 7 Jun 2016 08:10:58 -0700 Subject: [PATCH 02/16] Fix running --- .../main/java/io/druid/cli/CliTierFork.java | 30 ------------- .../java/io/druid/cli/CliTierForkTest.java | 42 ------------------- .../resources/TaskLogResourceTest.java | 2 +- 3 files changed, 1 insertion(+), 73 deletions(-) diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java index 57767f583e51..30d59e726284 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java @@ -89,8 +89,6 @@ protected List getModules() @Override public void configure(Binder binder) { - binder.bind(Task.class).toProvider(TaskProvider.class).in(LazySingleton.class); - binder.bind(PortWriter.class).in(ManageLifecycleLast.class); LifecycleModule.register(binder, PortWriter.class); @@ -206,31 +204,3 @@ abstract class ParentMonitorInputStreamFaker extends InputStream { } - -class TaskProvider implements Provider -{ - private final ObjectMapper jsonMapper; - private final File taskJson; - - @Inject - public TaskProvider( - @Json - ObjectMapper mapper, - ExecutorLifecycleConfig config - ) - { - jsonMapper = mapper; - this.taskJson = config.getTaskFile(); - } - - @Override - public Task get() - { - try { - return jsonMapper.readValue(taskJson, Task.class); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } -} diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java index bc2f83cd3dae..04a930ce250b 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java @@ -142,48 +142,6 @@ public void testParentMonitorInputStreamFakerProviderINTERRUPT() throws Exceptio } } - @Test - public void testTaskProvider() throws Exception - { - final Task task = new NoopTask("task_id", 0, 0, null, null, null); - final ObjectMapper mapper = EasyMock.createStrictMock(ObjectMapper.class); - EasyMock.expect(mapper.readValue(EasyMock.anyObject(File.class), EasyMock.eq(Task.class))).andReturn(task).once(); - EasyMock.replay(mapper); - final TaskProvider taskProvider = new TaskProvider(mapper, new ExecutorLifecycleConfig().setTaskFile( - temporaryFolder.newFile() - )); - Assert.assertEquals(task, taskProvider.get()); - } - - - @Test - public void testTaskProviderExceptional() throws Exception - { - final String msg = "test exception"; - final IOException ex = new IOException(msg); - expectedEx.expectCause(new BaseMatcher() - { - @Override - public void describeTo(Description description) - { - - } - - @Override - public boolean matches(Object o) - { - return ex == o; - } - }); - final ObjectMapper mapper = EasyMock.createStrictMock(ObjectMapper.class); - EasyMock.expect(mapper.readValue(EasyMock.anyObject(File.class), EasyMock.eq(Task.class))).andThrow(ex).once(); - EasyMock.replay(mapper); - final TaskProvider taskProvider = new TaskProvider(mapper, new ExecutorLifecycleConfig().setTaskFile( - temporaryFolder.newFile() - )); - taskProvider.get(); - } - @After public void tearDown() { diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java index b3a4fdc20ba9..fae7fd511d88 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java @@ -88,7 +88,7 @@ public void testNegativeOffset() throws Exception Assert.assertFalse(logFile.exists()); Assert.assertTrue(logFile.createNewFile()); final Response response = taskLogResource.getLog(-1); - Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); } @Test From f6fc79a63cac28bdbe72896b175d2689a4368e2c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 14 Jun 2016 10:53:52 -0700 Subject: [PATCH 03/16] More tests --- .../routing/DynamicUpstreamReporterTest.java | 14 +- .../RemoteTaskRunnerTierFactoryTest.java | 134 +++++++++++++++--- 2 files changed, 122 insertions(+), 26 deletions(-) diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java index d678e7fefbac..e18cb8ac00c8 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java @@ -32,13 +32,18 @@ import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceInstance; import org.easymock.EasyMock; +import org.easymock.EasyMockRunner; +import org.easymock.Mock; +import org.easymock.MockType; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Duration; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +@RunWith(EasyMockRunner.class) public class DynamicUpstreamReporterTest { private static final String DEFAULT_UPSTREAM = "upstream_service"; @@ -46,9 +51,12 @@ public class DynamicUpstreamReporterTest TaskStatusReporterModule.DynamicUpstreamReporter reporter = null; final TaskMaster taskMaster = EasyMock.createStrictMock(TaskMaster.class); final HttpClient httpClient = EasyMock.createStrictMock(HttpClient.class); - final ServiceDiscovery serviceDiscovery = EasyMock.createStrictMock(ServiceDiscovery.class); - final ServiceInstance service = EasyMock.createStrictMock(ServiceInstance.class); - final TaskRunnerReporter runner = EasyMock.createStrictMock(TaskRunnerReporter.class); + @Mock(type = MockType.STRICT) + ServiceDiscovery serviceDiscovery; + @Mock(type = MockType.STRICT) + ServiceInstance service; + @Mock(type = MockType.STRICT) + TaskRunnerReporter runner; final TaskStatus status = TaskStatus.success("task_id"); final diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java index be1f8fa848a6..485f4736480d 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java @@ -20,74 +20,162 @@ package io.druid.indexing.overlord.routing; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Supplier; import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.http.client.HttpClient; +import io.druid.concurrent.Execs; +import io.druid.indexing.overlord.RemoteTaskRunner; +import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerResourceManagementConfig; import io.druid.indexing.overlord.autoscaling.ResourceManagementSchedulerConfig; import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.jackson.DefaultObjectMapper; import io.druid.server.initialization.IndexerZkConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.easymock.EasyMock; +import org.easymock.EasyMockRunner; +import org.easymock.Mock; +import org.easymock.MockType; +import org.junit.After; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import java.util.concurrent.ScheduledExecutorService; + +@RunWith(EasyMockRunner.class) public class RemoteTaskRunnerTierFactoryTest { - final RemoteTaskRunnerConfig remoteTaskRunnerConfig = EasyMock.createStrictMock(RemoteTaskRunnerConfig.class); - final PendingTaskBasedWorkerResourceManagementConfig config = EasyMock.createStrictMock( - PendingTaskBasedWorkerResourceManagementConfig.class); - final Supplier workerConfigRef = Suppliers.ofInstance(EasyMock.createStrictMock( - WorkerBehaviorConfig.class)); - final CuratorFramework curator = EasyMock.createStrictMock(CuratorFramework.class); - final IndexerZkConfig zkPaths = EasyMock.createStrictMock(IndexerZkConfig.class); + @Mock(type = MockType.STRICT) + PendingTaskBasedWorkerResourceManagementConfig config; + + @Mock(type = MockType.STRICT) + WorkerBehaviorConfig workerBehaviorConfig; + + @Mock(type = MockType.STRICT) + CuratorFramework curator; + + @Mock(type = MockType.STRICT) + HttpClient httpClient; + + @Mock(type = MockType.STRICT) + ResourceManagementSchedulerConfig resourceManagementSchedulerConfig; + + final ScheduledExecutorService scheduledExecutorService = Execs.scheduledSingleThreaded("TestThread"); + final ScheduledExecutorFactory executorFactory = new ScheduledExecutorFactory() + { + @Override + public ScheduledExecutorService create(int corePoolSize, String nameFormat) + { + return scheduledExecutorService; + } + }; + IndexerZkConfig zkPaths = new IndexerZkConfig(new ZkPathsConfig(), null, null, null, null, null); final ObjectMapper jsonMapper = new DefaultObjectMapper(); - final HttpClient httpClient = EasyMock.createStrictMock(HttpClient.class); - final ScheduledExecutorFactory executorFactory = EasyMock.createStrictMock(ScheduledExecutorFactory.class); - final ResourceManagementSchedulerConfig resourceManagementSchedulerConfig = - EasyMock.createStrictMock(ResourceManagementSchedulerConfig.class); - final RemoteTaskRunnerTierFactory factory = new RemoteTaskRunnerTierFactory( - remoteTaskRunnerConfig, - config, - workerConfigRef, - curator, - zkPaths, - jsonMapper, - httpClient, - executorFactory, - resourceManagementSchedulerConfig - ); + final RemoteTaskRunnerConfig remoteTaskRunnerConfig = new RemoteTaskRunnerConfig(); @Test public void testBuild() throws Exception { + final RemoteTaskRunnerTierFactory factory = new RemoteTaskRunnerTierFactory( + remoteTaskRunnerConfig, + config, + Suppliers.ofInstance(workerBehaviorConfig), + curator, + zkPaths, + jsonMapper, + httpClient, + executorFactory, + resourceManagementSchedulerConfig + ); + EasyMock.expect(resourceManagementSchedulerConfig.isDoAutoscale()).andReturn(false).times(2); + EasyMock.replay(resourceManagementSchedulerConfig); + final TaskRunner runner = factory.build(); + Assert.assertTrue(runner instanceof RemoteTaskRunner); + final RemoteTaskRunner rtr = (RemoteTaskRunner) runner; + Assert.assertEquals(ImmutableList.of(), rtr.getKnownTasks()); + Assert.assertEquals(ImmutableList.of(), rtr.getLazyWorkers()); + Assert.assertEquals(ImmutableList.of(), rtr.getPendingTaskPayloads()); + Assert.assertEquals(ImmutableList.of(), rtr.getPendingTasks()); + Assert.assertEquals(ImmutableList.of(), rtr.getRunningTasks()); + Assert.assertEquals(ImmutableList.of(), rtr.getWorkers()); } @Test public void testGetRemoteTaskRunnerConfig() throws Exception { + final RemoteTaskRunnerTierFactory factory = new RemoteTaskRunnerTierFactory( + remoteTaskRunnerConfig, + config, + Suppliers.ofInstance(workerBehaviorConfig), + curator, + zkPaths, + jsonMapper, + httpClient, + executorFactory, + resourceManagementSchedulerConfig + ); Assert.assertEquals(remoteTaskRunnerConfig, factory.getRemoteTaskRunnerConfig()); } @Test public void testGetZkPaths() throws Exception { + final RemoteTaskRunnerTierFactory factory = new RemoteTaskRunnerTierFactory( + remoteTaskRunnerConfig, + config, + Suppliers.ofInstance(workerBehaviorConfig), + curator, + zkPaths, + jsonMapper, + httpClient, + executorFactory, + resourceManagementSchedulerConfig + ); Assert.assertEquals(zkPaths, factory.getZkPaths()); } @Test public void testGetPendingConfig() throws Exception { + final RemoteTaskRunnerTierFactory factory = new RemoteTaskRunnerTierFactory( + remoteTaskRunnerConfig, + config, + Suppliers.ofInstance(workerBehaviorConfig), + curator, + zkPaths, + jsonMapper, + httpClient, + executorFactory, + resourceManagementSchedulerConfig + ); Assert.assertEquals(config, factory.getPendingTaskBasedWorkerResourceManagementConfig()); } @Test public void testGetResourceManagementSchedulerConfig() throws Exception { + final RemoteTaskRunnerTierFactory factory = new RemoteTaskRunnerTierFactory( + remoteTaskRunnerConfig, + config, + Suppliers.ofInstance(workerBehaviorConfig), + curator, + zkPaths, + jsonMapper, + httpClient, + executorFactory, + resourceManagementSchedulerConfig + ); Assert.assertEquals(remoteTaskRunnerConfig, factory.getRemoteTaskRunnerConfig()); } + + @After + public void tearDown() + { + scheduledExecutorService.shutdownNow(); + } } From 5f22c47c0b795514c368d70b359535d668f147c6 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 14 Jun 2016 14:22:48 -0700 Subject: [PATCH 04/16] Fix tests --- .../test/java/io/druid/cli/CliTierForkTest.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java index 04a930ce250b..5a341a8dd27b 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java @@ -31,17 +31,15 @@ import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.RemoteChatHandler; import io.druid.guice.annotations.Self; -import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.BusyTask; import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; import io.druid.indexing.overlord.resources.DeadhandResource; -import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; import io.druid.initialization.Initialization; +import io.druid.jackson.DefaultObjectMapper; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; import org.easymock.EasyMock; -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; import org.junit.After; import org.junit.Assert; import org.junit.Rule; @@ -64,9 +62,14 @@ public class CliTierForkTest @Test public void testGetModules() throws Exception { + final File lockFile = temporaryFolder.newFile(); + Assert.assertTrue(lockFile.delete() || !lockFile.exists()); + Assert.assertFalse(lockFile.exists()); + final Task task = new BusyTask("taskId", lockFile.toString(), 100); + final CliTierFork cliTierFork = new CliTierFork(); cliTierFork.taskAndStatusFile = ImmutableList.of( - temporaryFolder.newFile().toString(), + temporaryFolder.newFile().toString(), // Should not actually read from this temporaryFolder.newFile().toString() ); final Injector startupInjector = GuiceInjectors.makeStartupInjector(); @@ -94,6 +97,7 @@ public void configure(Binder binder) Key.get(ServerConfig.class, RemoteChatHandler.class), new ServerConfig() ); + binder.bind(Task.class).toInstance(task); } }) ) From 726e2bdb89b4caaf927a301ea2fbcc6ee202e45c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 14 Jun 2016 14:23:17 -0700 Subject: [PATCH 05/16] Reformat code --- .../tasktier/src/main/java/io/druid/cli/CliTierFork.java | 1 - .../tasktier/src/test/java/io/druid/cli/CliTierForkTest.java | 2 -- 2 files changed, 3 deletions(-) diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java index 30d59e726284..365187152331 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java @@ -39,7 +39,6 @@ import io.druid.guice.annotations.Self; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; -import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.PortWriter; import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java index 5a341a8dd27b..456d8da15dac 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java @@ -19,7 +19,6 @@ package io.druid.cli; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Binding; @@ -36,7 +35,6 @@ import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; import io.druid.indexing.overlord.resources.DeadhandResource; import io.druid.initialization.Initialization; -import io.druid.jackson.DefaultObjectMapper; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; import org.easymock.EasyMock; From b172a3aac99b4e155efb42c0b020510d4ba6c235 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 14 Jun 2016 15:07:58 -0700 Subject: [PATCH 06/16] Update version --- extensions-core/tasktier/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/tasktier/pom.xml b/extensions-core/tasktier/pom.xml index 732b71fc55bb..622a5b4385d1 100644 --- a/extensions-core/tasktier/pom.xml +++ b/extensions-core/tasktier/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.9.1-SNAPSHOT + 0.9.2-SNAPSHOT ../../pom.xml From 3a650e7a3903feceb1937d2384e55c93b4cbb9b2 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 14 Jun 2016 16:44:20 -0700 Subject: [PATCH 07/16] Checkpoint --- distribution/pom.xml | 2 ++ indexing-service/pom.xml | 5 +++++ server/pom.xml | 5 +++++ services/pom.xml | 5 +++++ 4 files changed, 17 insertions(+) diff --git a/distribution/pom.xml b/distribution/pom.xml index a1db657935a3..8893a561841f 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -94,6 +94,8 @@ io.druid.extensions:mysql-metadata-storage -c io.druid.extensions:postgresql-metadata-storage + -c + io.druid.extensions:druid-tasktier diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index aa1cd847fd2c..ca54c95a6e33 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -30,6 +30,11 @@ + + io.druid + druid-api + ${project.parent.version} + io.druid druid-common diff --git a/server/pom.xml b/server/pom.xml index fc8c388f8ac3..89ab3e2622b3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -30,6 +30,11 @@ + + io.druid + druid-api + ${project.parent.version} + io.druid druid-processing diff --git a/services/pom.xml b/services/pom.xml index 44ff8cf994b1..47e01e9e0c52 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -31,6 +31,11 @@ + + io.druid + druid-api + ${project.parent.version} + io.druid druid-common From 37fc2b01b3d533ca3cfea915d541fc1fb235cb29 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 14 Jun 2016 20:19:11 -0700 Subject: [PATCH 08/16] Add better tests for module --- .../TierRoutingTaskRunnerFactory.java | 2 + .../routing/DynamicUpstreamReporter.java | 89 ++++++++++++++ ...askStatusPostToLeaderListenerResource.java | 2 +- .../TaskStatusPostToLeaderReporter.java | 2 +- ...eporterModule.java => TaskTierModule.java} | 114 ++++-------------- .../io.druid.initialization.DruidModule | 2 +- .../routing/DynamicUpstreamReporterTest.java | 6 +- .../routing/TaskStatusReporterModuleTest.java | 48 -------- .../overlord/routing/TaskTierModuleTest.java | 100 +++++++++++++++ .../main/java/io/druid/cli/CliOverlord.java | 2 +- 10 files changed, 223 insertions(+), 144 deletions(-) create mode 100644 extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporter.java rename extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/{TaskStatusReporterModule.java => TaskTierModule.java} (50%) delete mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusReporterModuleTest.java create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskTierModuleTest.java diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java index 7febb8cf0574..4e3441452734 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java @@ -19,6 +19,8 @@ package io.druid.indexing.overlord; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Supplier; import com.google.inject.Inject; import com.metamx.common.concurrent.ScheduledExecutorFactory; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporter.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporter.java new file mode 100644 index 000000000000..68fb0c79803a --- /dev/null +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporter.java @@ -0,0 +1,89 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.google.common.base.Optional; +import com.google.inject.Inject; +import com.google.inject.name.Named; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import io.druid.guice.annotations.Global; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskRunner; +import org.apache.curator.x.discovery.ServiceDiscovery; + +public class DynamicUpstreamReporter implements TaskStatusReporter +{ + private static final Logger LOG = new Logger(DynamicUpstreamReporter.class); + @Inject(optional = true) + private TaskMaster taskMaster = null; + @Inject + private + @Global + HttpClient httpClient = null; + @Inject + private ServiceDiscovery discovery = null; + @Inject + private + @Named(TaskTierModule.UPSTREAM_SERVICE_NAME_CONSTANT_KEY) + String upstreamService = null; + + public DynamicUpstreamReporter() {} + + public DynamicUpstreamReporter( + TaskMaster taskMaster, + @Global HttpClient httpClient, + ServiceDiscovery discovery, + @Named(TaskTierModule.UPSTREAM_SERVICE_NAME_CONSTANT_KEY) String upstreamService + ) + { + this.taskMaster = taskMaster; + this.httpClient = httpClient; + this.discovery = discovery; + this.upstreamService = upstreamService; + } + + @Override + public boolean reportStatus(TaskStatus status) + { + TaskStatusReporter reporter = new TaskStatusPostToLeaderReporter(httpClient, discovery, upstreamService); + if (taskMaster != null) { + final Optional taskRunnerOptional = taskMaster.getTaskRunner(); + if (taskRunnerOptional.isPresent()) { + final TaskRunner runner = taskRunnerOptional.get(); + if (runner instanceof TaskStatusReporter) { + reporter = (TaskStatusReporter) runner; + } else { + LOG.debug( + "Expected [%s] but was [%s]. Trying POST", + TaskStatusReporter.class, + runner.getClass().getCanonicalName() + ); + } + } else { + LOG.debug("No task runner. Trying POST"); + } + } else { + LOG.debug("No task master. Trying POST"); + } + return reporter.reportStatus(status); + } +} diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResource.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResource.java index 04b800e9b7c3..27c6af2ba4e4 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResource.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderListenerResource.java @@ -43,7 +43,7 @@ public class TaskStatusPostToLeaderListenerResource @Inject public TaskStatusPostToLeaderListenerResource( - @Named(TaskStatusReporterModule.UPSTREAM_TASK_REPORTER_NAME) TaskStatusReporter upstreamReporter + @Named(TaskTierModule.UPSTREAM_TASK_REPORTER_NAME) TaskStatusReporter upstreamReporter ) { this.upstreamReporter = upstreamReporter; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java index 7e981905457e..4614738420c0 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java @@ -59,7 +59,7 @@ public class TaskStatusPostToLeaderReporter implements TaskStatusReporter public TaskStatusPostToLeaderReporter( @Global HttpClient httpClient, ServiceDiscovery discovery, - @Named(TaskStatusReporterModule.UPSTREAM_SERVICE_NAME_CONSTANT_KEY) String upstreamService + @Named(TaskTierModule.UPSTREAM_SERVICE_NAME_CONSTANT_KEY) String upstreamService ) { this.httpClient = httpClient; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporterModule.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskTierModule.java similarity index 50% rename from extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporterModule.java rename to extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskTierModule.java index 7476d7be5773..59e3a3016656 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusReporterModule.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskTierModule.java @@ -21,36 +21,27 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.Module; -import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Key; import com.google.inject.Provider; -import com.google.inject.name.Named; import com.google.inject.name.Names; -import com.metamx.common.logger.Logger; -import com.metamx.http.client.HttpClient; import io.druid.client.indexing.IndexingServiceSelectorConfig; import io.druid.guice.JacksonConfigProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.PolyBind; -import io.druid.guice.annotations.Global; -import io.druid.indexing.common.TaskStatus; -import io.druid.indexing.overlord.TaskMaster; -import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.overlord.TaskRunnerFactory; import io.druid.indexing.overlord.TierRoutingTaskRunnerFactory; import io.druid.initialization.DruidModule; -import org.apache.curator.x.discovery.ServiceDiscovery; +import io.druid.server.initialization.jetty.JettyBindings; import java.util.List; -public class TaskStatusReporterModule implements DruidModule +public class TaskTierModule implements DruidModule { - private static final Logger LOG = new Logger(TaskStatusReporterModule.class); public static final String UPSTREAM_SERVICE_NAME_CONSTANT_KEY = "upstreamServiceName"; public static final String UPSTREAM_TASK_REPORTER_NAME = "upstream"; public static final String UPSTREAM_PROPERTY_KEY = "io.druid.index.tier.upstreamServiceName"; @@ -70,8 +61,10 @@ public void configure(Binder binder) binder, Key.get(TaskRunnerFactory.class) ).addBinding(POLYBIND_ROUTING_KEY).to(TierRoutingTaskRunnerFactory.class); + binder.bind(TierRoutingTaskRunnerFactory.class).in(LazySingleton.class); Jerseys.addResource(binder, TaskStatusPostToLeaderListenerResource.class); + JettyBindings.addQosFilter(binder, TaskStatusPostToLeaderListenerResource.PATH, 2); binder.bind(Key.get(TaskStatusReporter.class, Names.named(UPSTREAM_TASK_REPORTER_NAME))) .to(DynamicUpstreamReporter.class) @@ -81,91 +74,34 @@ public void configure(Binder binder) .toProvider(UpstreamNameHolderProvider.class); JacksonConfigProvider.bind(binder, ROUTING_CONFIG_KEY, TierRouteConfig.class, null); } +} - public static class UpstreamNameHolder - { - @JsonProperty - String upstreamServiceName = IndexingServiceSelectorConfig.DEFAULT_SERVICE_NAME; - - public String getUpstreamServiceName() - { - return upstreamServiceName; - } - } +class UpstreamNameHolder +{ + @JsonProperty + String upstreamServiceName = IndexingServiceSelectorConfig.DEFAULT_SERVICE_NAME; - public static class UpstreamNameHolderProvider implements Provider + public String getUpstreamServiceName() { - private final UpstreamNameHolder upstreamNameHolder; + return upstreamServiceName; + } +} - @Inject - public UpstreamNameHolderProvider( - UpstreamNameHolder upstreamNameHolder - ) - { - this.upstreamNameHolder = upstreamNameHolder; - } +class UpstreamNameHolderProvider implements Provider +{ + private final UpstreamNameHolder upstreamNameHolder; - @Override - public String get() - { - return upstreamNameHolder.getUpstreamServiceName(); - } + @Inject + public UpstreamNameHolderProvider( + UpstreamNameHolder upstreamNameHolder + ) + { + this.upstreamNameHolder = upstreamNameHolder; } - public static class DynamicUpstreamReporter implements TaskStatusReporter + @Override + public String get() { - @Inject(optional = true) - private TaskMaster taskMaster = null; - @Inject - private - @Global - HttpClient httpClient = null; - @Inject - private ServiceDiscovery discovery = null; - @Inject - private - @Named(UPSTREAM_SERVICE_NAME_CONSTANT_KEY) - String upstreamService = null; - - public DynamicUpstreamReporter() {} - - public DynamicUpstreamReporter( - TaskMaster taskMaster, - @Global HttpClient httpClient, - ServiceDiscovery discovery, - @Named(UPSTREAM_SERVICE_NAME_CONSTANT_KEY) String upstreamService - ) - { - this.taskMaster = taskMaster; - this.httpClient = httpClient; - this.discovery = discovery; - this.upstreamService = upstreamService; - } - - @Override - public boolean reportStatus(TaskStatus status) - { - TaskStatusReporter reporter = new TaskStatusPostToLeaderReporter(httpClient, discovery, upstreamService); - if (taskMaster != null) { - final Optional taskRunnerOptional = taskMaster.getTaskRunner(); - if (taskRunnerOptional.isPresent()) { - final TaskRunner runner = taskRunnerOptional.get(); - if (runner instanceof TaskStatusReporter) { - reporter = (TaskStatusReporter) runner; - } else { - LOG.debug( - "Expected [%s] but was [%s]. Trying POST", - TaskStatusReporter.class, - runner.getClass().getCanonicalName() - ); - } - } else { - LOG.debug("No task runner. Trying POST"); - } - } else { - LOG.debug("No task master. Trying POST"); - } - return reporter.reportStatus(status); - } + return upstreamNameHolder.getUpstreamServiceName(); } } diff --git a/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.initialization.DruidModule index 3a3920c22156..f629c8d8a4d1 100644 --- a/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/extensions-core/tasktier/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -18,4 +18,4 @@ # io.druid.indexing.overlord.config.TierConfigModule -io.druid.indexing.overlord.routing.TaskStatusReporterModule +io.druid.indexing.overlord.routing.TaskTierModule diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java index e18cb8ac00c8..94f50dad714a 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporterTest.java @@ -48,7 +48,7 @@ public class DynamicUpstreamReporterTest { private static final String DEFAULT_UPSTREAM = "upstream_service"; final Object memoryBarrier = new Object(); - TaskStatusReporterModule.DynamicUpstreamReporter reporter = null; + DynamicUpstreamReporter reporter = null; final TaskMaster taskMaster = EasyMock.createStrictMock(TaskMaster.class); final HttpClient httpClient = EasyMock.createStrictMock(HttpClient.class); @Mock(type = MockType.STRICT) @@ -69,7 +69,7 @@ public void setUp() throws Exception EasyMock.expect(runner.reportStatus(EasyMock.eq(status))).andReturn(true).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(runner)).anyTimes(); EasyMock.replay(serviceDiscovery, taskMaster, httpClient, runner); - reporter = new TaskStatusReporterModule.DynamicUpstreamReporter( + reporter = new DynamicUpstreamReporter( taskMaster, httpClient, serviceDiscovery, @@ -109,7 +109,7 @@ public void testReportStatusSillyRunner() throws Exception @Test public void testReportStatusNullManager() throws Exception { - reporter = new TaskStatusReporterModule.DynamicUpstreamReporter( + reporter = new DynamicUpstreamReporter( null, httpClient, serviceDiscovery, diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusReporterModuleTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusReporterModuleTest.java deleted file mode 100644 index 0142b735fd33..000000000000 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskStatusReporterModuleTest.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.indexing.overlord.routing; - -import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; -import org.junit.Test; - -public class TaskStatusReporterModuleTest -{ - @Test - public void testUpstreamNameHolder() throws Exception - { - final String name = "foo"; - final TaskStatusReporterModule.UpstreamNameHolder upstreamNameHolder = new TaskStatusReporterModule.UpstreamNameHolder(); - upstreamNameHolder.upstreamServiceName = name; - Assert.assertEquals(name, upstreamNameHolder.getUpstreamServiceName()); - final ObjectMapper mapper = new DefaultObjectMapper(); - Assert.assertEquals( - name, - mapper.readValue( - mapper.writeValueAsString(upstreamNameHolder), - TaskStatusReporterModule.UpstreamNameHolder.class - ).getUpstreamServiceName() - ); - final TaskStatusReporterModule.UpstreamNameHolderProvider provider = new TaskStatusReporterModule.UpstreamNameHolderProvider( - upstreamNameHolder); - Assert.assertEquals(name, provider.get()); - } -} diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskTierModuleTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskTierModuleTest.java new file mode 100644 index 000000000000..507925994166 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskTierModuleTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.routing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.name.Names; +import io.druid.cli.CliOverlord; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.JsonConfigurator; +import io.druid.indexing.overlord.TaskRunnerFactory; +import io.druid.indexing.overlord.TierRoutingTaskRunner; +import io.druid.indexing.overlord.TierRoutingTaskRunnerFactory; +import io.druid.initialization.Initialization; +import io.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Properties; +import java.util.UUID; + +public class TaskTierModuleTest +{ + @Test + public void testUpstreamNameHolder() throws Exception + { + final String name = "foo"; + final TaskTierModule.UpstreamNameHolder upstreamNameHolder = new TaskTierModule.UpstreamNameHolder(); + upstreamNameHolder.upstreamServiceName = name; + Assert.assertEquals(name, upstreamNameHolder.getUpstreamServiceName()); + final ObjectMapper mapper = new DefaultObjectMapper(); + Assert.assertEquals( + name, + mapper.readValue( + mapper.writeValueAsString(upstreamNameHolder), + TaskTierModule.UpstreamNameHolder.class + ).getUpstreamServiceName() + ); + final TaskTierModule.UpstreamNameHolderProvider provider = new TaskTierModule.UpstreamNameHolderProvider( + upstreamNameHolder); + Assert.assertEquals(name, provider.get()); + } + + @Test + public void testTaskRunnerFactoryBinding() + { + final String propertyPrefix = "druid.indexer.runner.type"; + final Collection modules = new ArrayList<>(); + modules.addAll(new CliOverlord().getModules()); + final Injector injector = Initialization.makeInjectorWithModules(GuiceInjectors.makeStartupInjector(), modules); + final Properties properties = injector.getInstance(Properties.class); + properties.setProperty(propertyPrefix, TaskTierModule.POLYBIND_ROUTING_KEY); + final TaskRunnerFactory factory = injector.getInstance(TaskRunnerFactory.class); + final TierRoutingTaskRunnerFactory tierRoutingTaskRunnerFactory = (TierRoutingTaskRunnerFactory)factory; + final TierRoutingTaskRunner runner = tierRoutingTaskRunnerFactory.build(); + Assert.assertEquals(ImmutableList.of(), runner.getKnownTasks()); + Assert.assertEquals(ImmutableList.of(), runner.getPendingTasks()); + Assert.assertEquals(ImmutableList.of(), runner.getRunningTasks()); + } + + @Test + public void testTierRouteConfigBinding() + { + final Injector injector = Initialization.makeInjectorWithModules(GuiceInjectors.makeStartupInjector(), ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + + } + } + )); + final TierRouteConfig routeConfig; + } +} diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index d0545cf7444b..5e7f83fe1445 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -199,7 +199,7 @@ private void configureRunners(Binder binder) ); IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder); - biddy.addBinding("local").to(ForkingTaskRunnerFactory.class); + biddy.addBinding("local").to(ForkingTaskRunnerFactory.class).in(LazySingleton.class); binder.bind(ForkingTaskRunnerFactory.class).in(LazySingleton.class); biddy.addBinding(RemoteTaskRunnerFactory.TYPE_NAME).to(RemoteTaskRunnerFactory.class).in(LazySingleton.class); From 2b065500a0aafcdf4dec21c0902aac6b2006f5c4 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 15 Jun 2016 08:59:59 -0700 Subject: [PATCH 09/16] Fix test --- .../overlord/routing/TaskTierModuleTest.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskTierModuleTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskTierModuleTest.java index 507925994166..28a1a418cfa8 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskTierModuleTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TaskTierModuleTest.java @@ -27,14 +27,17 @@ import com.google.inject.Module; import com.google.inject.name.Names; import io.druid.cli.CliOverlord; +import io.druid.common.config.JacksonConfigManager; import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.JsonConfigurator; +import io.druid.guice.annotations.Self; import io.druid.indexing.overlord.TaskRunnerFactory; import io.druid.indexing.overlord.TierRoutingTaskRunner; import io.druid.indexing.overlord.TierRoutingTaskRunnerFactory; import io.druid.initialization.Initialization; import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.DruidNode; import org.junit.Assert; import org.junit.Test; @@ -49,7 +52,7 @@ public class TaskTierModuleTest public void testUpstreamNameHolder() throws Exception { final String name = "foo"; - final TaskTierModule.UpstreamNameHolder upstreamNameHolder = new TaskTierModule.UpstreamNameHolder(); + final UpstreamNameHolder upstreamNameHolder = new UpstreamNameHolder(); upstreamNameHolder.upstreamServiceName = name; Assert.assertEquals(name, upstreamNameHolder.getUpstreamServiceName()); final ObjectMapper mapper = new DefaultObjectMapper(); @@ -57,10 +60,10 @@ public void testUpstreamNameHolder() throws Exception name, mapper.readValue( mapper.writeValueAsString(upstreamNameHolder), - TaskTierModule.UpstreamNameHolder.class + UpstreamNameHolder.class ).getUpstreamServiceName() ); - final TaskTierModule.UpstreamNameHolderProvider provider = new TaskTierModule.UpstreamNameHolderProvider( + final UpstreamNameHolderProvider provider = new UpstreamNameHolderProvider( upstreamNameHolder); Assert.assertEquals(name, provider.get()); } @@ -91,10 +94,13 @@ public void testTierRouteConfigBinding() @Override public void configure(Binder binder) { - + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null) + ); } } )); + final JacksonConfigManager jacksonConfigManager = injector.getInstance(JacksonConfigManager.class); final TierRouteConfig routeConfig; } } From 6cd0ac2a6a973d6368b214145cfcd44998d32c92 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 15 Jun 2016 12:11:04 -0700 Subject: [PATCH 10/16] Add lifecycle to runner --- .../io/druid/indexing/overlord/TierRoutingTaskRunner.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java index a79ce4eaaeb8..5430b375302a 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java @@ -30,6 +30,8 @@ import com.metamx.common.IAE; import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; @@ -170,12 +172,14 @@ public Optional getScalingStats() } @Override + @LifecycleStart public void start() { managementStrategy.startManagement(null); } @Override + @LifecycleStop public void stop() { managementStrategy.stopManagement(); From d813082c636126276fc457c893fbd6e1c672885e Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 20 Jun 2016 15:48:40 -0700 Subject: [PATCH 11/16] checkpoint --- .../io/druid/indexing/overlord/routing/TierRouteConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierRouteConfig.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierRouteConfig.java index 90818fdfc37b..c8e4861103bf 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierRouteConfig.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TierRouteConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; @@ -32,7 +33,7 @@ public class TierRouteConfig { - // Fields exposed for unit tests + @VisibleForTesting @JsonProperty("tierMap") Map> tierMap = ImmutableMap.of(); @@ -40,8 +41,7 @@ public class TierRouteConfig ObjectMapper mapper = null; public - @NotNull - TierTaskRunnerFactory getRouteFactory(@NotNull String tier) + TierTaskRunnerFactory getRouteFactory(String tier) { final Map map = tierMap.get(Preconditions.checkNotNull(tier, "tier")); if (map == null) { From 6962d49715e09d951d431bf01499f69ba61150f8 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 22 Jun 2016 12:36:22 -0700 Subject: [PATCH 12/16] Checkpoint --- .../overlord/config/TierConfigModule.java | 80 +++++++++++ .../overlord/routing/TaskTierModule.java | 3 - .../config/TierConfigResourceTest.java | 126 ++++++++++++++++++ 3 files changed, 206 insertions(+), 3 deletions(-) create mode 100644 extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/config/TierConfigResourceTest.java diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java index 9fe379361713..eca0760557e6 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java @@ -23,6 +23,14 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; +import com.google.inject.Inject; +import com.metamx.common.logger.Logger; +import io.druid.audit.AuditInfo; +import io.druid.audit.AuditManager; +import io.druid.common.config.JacksonConfigManager; +import io.druid.common.utils.ServletResourceUtils; +import io.druid.guice.JacksonConfigProvider; +import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.indexing.overlord.TierTaskDiscovery; @@ -30,10 +38,24 @@ import io.druid.indexing.overlord.routing.TierTaskRunnerFactory; import io.druid.initialization.DruidModule; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.HeaderParam; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; public class TierConfigModule implements DruidModule { + static final String ROUTE_CONFIG_KEY = "druid.tier.route.config"; + @Override public List getJacksonModules() { @@ -50,6 +72,64 @@ public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.indexer.runner", TierLocalTaskRunnerConfig.class); JsonConfigProvider.bind(binder, "druid.zk.paths", TierForkZkConfig.class); + Jerseys.addResource(binder, TierConfigResource.class); binder.bind(TierTaskDiscovery.class).in(LazySingleton.class); + JacksonConfigProvider.bind(binder, ROUTE_CONFIG_KEY, TierRouteConfig.class, null); + } +} + +@Path("/druid/tier/v1/config") +class TierConfigResource +{ + private static final Logger LOG = new Logger(TierConfigResource.class); + private final JacksonConfigManager configManager; + private final AtomicReference routeConfigRef; + + @Inject + public TierConfigResource( + final JacksonConfigManager configManager + ) + { + this.configManager = configManager; + routeConfigRef = configManager.watch(TierConfigModule.ROUTE_CONFIG_KEY, TierRouteConfig.class, null); + } + + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response updateConfig( + final TierRouteConfig tierRouteConfig, + @HeaderParam(AuditManager.X_DRUID_AUTHOR) @DefaultValue("") final String author, + @HeaderParam(AuditManager.X_DRUID_COMMENT) @DefaultValue("") final String comment, + @Context final HttpServletRequest req + ) + { + if (!configManager.set( + TierConfigModule.ROUTE_CONFIG_KEY, + tierRouteConfig, + new AuditInfo(author, comment, req.getRemoteHost()) + )) { + LOG.debug("Unable to set %s from [%s]", tierRouteConfig, req.getRemoteHost()); + return Response.status(Response.Status.BAD_REQUEST).build(); + } + LOG.info("Updated tier route config per request from %s@%s: %s", author, req.getRemoteHost(), comment); + return Response.status(Response.Status.ACCEPTED).build(); + } + + @GET + @Produces(MediaType.APPLICATION_JSON) + public Response getConfig( + @Context final HttpServletRequest req + ) + { + final TierRouteConfig routeConfig = routeConfigRef.get(); + if (routeConfig == null) { + LOG.debug("Requested config from [%s] but no config is set.", req.getRemoteHost()); + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.sanitizeException(new IllegalArgumentException("No config set"))) + .build(); + } + LOG.debug("Returning config to [%s]", req.getRemoteHost()); + return Response.ok(routeConfig).build(); } } diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskTierModule.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskTierModule.java index 59e3a3016656..d2803c113fe5 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskTierModule.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskTierModule.java @@ -28,7 +28,6 @@ import com.google.inject.Provider; import com.google.inject.name.Names; import io.druid.client.indexing.IndexingServiceSelectorConfig; -import io.druid.guice.JacksonConfigProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; @@ -46,7 +45,6 @@ public class TaskTierModule implements DruidModule public static final String UPSTREAM_TASK_REPORTER_NAME = "upstream"; public static final String UPSTREAM_PROPERTY_KEY = "io.druid.index.tier.upstreamServiceName"; public static final String POLYBIND_ROUTING_KEY = "routing"; - public static final String ROUTING_CONFIG_KEY = "druid.tier.routing.config"; @Override public List getJacksonModules() @@ -72,7 +70,6 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, UPSTREAM_PROPERTY_KEY, UpstreamNameHolder.class); binder.bind(Key.get(String.class, Names.named(UPSTREAM_SERVICE_NAME_CONSTANT_KEY))) .toProvider(UpstreamNameHolderProvider.class); - JacksonConfigProvider.bind(binder, ROUTING_CONFIG_KEY, TierRouteConfig.class, null); } } diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/config/TierConfigResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/config/TierConfigResourceTest.java new file mode 100644 index 000000000000..3f5ef3e1a192 --- /dev/null +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/config/TierConfigResourceTest.java @@ -0,0 +1,126 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord.config; + +import io.druid.audit.AuditInfo; +import io.druid.common.config.JacksonConfigManager; +import io.druid.indexing.overlord.routing.TierRouteConfig; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.core.Response; +import java.util.concurrent.atomic.AtomicReference; + +public class TierConfigResourceTest +{ + private static final String HOST = "localhost"; + private static final String AUTHOR = "some_author"; + private static final String COMMENT = "some_comment"; + + + private final AtomicReference configRef = new AtomicReference<>(null); + private JacksonConfigManager configManager; + private HttpServletRequest request; + private TierConfigResource resource; + + @Before + public void setUp() + { + configManager = prepareManager(configRef); + request = prepareRequest(); + EasyMock.replay(request, configManager); + resource = new TierConfigResource(configManager); + } + + @After + public void tearDown() + { + EasyMock.verify(request, configManager); + } + + @Test + public void testGetNoConfig() + { + Assert.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resource.getConfig(request).getStatus()); + } + + @Test + public void testGetWithConfig() + { + final TierRouteConfig tierRouteConfig = new TierRouteConfig(); + configRef.set(tierRouteConfig); + Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.getConfig(request).getStatus()); + Assert.assertEquals(tierRouteConfig, resource.getConfig(request).getEntity()); + } + + @Test + public void testSetConfigSuccess() + { + final TierRouteConfig otherTierRouteConfig = new TierRouteConfig(); + EasyMock.reset(configManager); + EasyMock.expect(configManager.set( + EasyMock.eq(TierConfigModule.ROUTE_CONFIG_KEY), + EasyMock.eq(otherTierRouteConfig), + EasyMock.eq(new AuditInfo(AUTHOR, COMMENT, request.getRemoteHost())) + )).andReturn(true).once(); + EasyMock.replay(configManager); + + final Response response = resource.updateConfig(otherTierRouteConfig, "some_author", "some_comment", request); + Assert.assertEquals(Response.Status.ACCEPTED.getStatusCode(), response.getStatus()); + } + + @Test + public void testSetConfigFailure() + { + final TierRouteConfig otherTierRouteConfig = new TierRouteConfig(); + EasyMock.reset(configManager); + EasyMock.expect(configManager.set( + EasyMock.eq(TierConfigModule.ROUTE_CONFIG_KEY), + EasyMock.eq(otherTierRouteConfig), + EasyMock.eq(new AuditInfo(AUTHOR, COMMENT, request.getRemoteHost())) + )).andReturn(false).once(); + EasyMock.replay(configManager); + + final Response response = resource.updateConfig(otherTierRouteConfig, "some_author", "some_comment", request); + Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); + } + + private JacksonConfigManager prepareManager(AtomicReference configRef) + { + final JacksonConfigManager configManager = EasyMock.createStrictMock(JacksonConfigManager.class); + EasyMock.expect(configManager.watch( + EasyMock.eq(TierConfigModule.ROUTE_CONFIG_KEY), + EasyMock.eq(TierRouteConfig.class), + EasyMock.isNull() + )).andReturn(configRef).once(); + return configManager; + } + + private HttpServletRequest prepareRequest() + { + final HttpServletRequest request = EasyMock.createStrictMock(HttpServletRequest.class); + EasyMock.expect(request.getRemoteHost()).andReturn("localhost").anyTimes(); + return request; + } +} \ No newline at end of file From 4d2792ff8e6265feeb9d9a065bd35ea2bc4f9e1d Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 23 Jun 2016 14:13:11 -0700 Subject: [PATCH 13/16] Fix sync in tier routing --- .../overlord/TierRoutingTaskRunner.java | 15 +- .../TierRoutingManagementStrategy.java | 347 +++++++++++------- .../TierRoutingManagementStrategyTest.java | 112 +++--- 3 files changed, 278 insertions(+), 196 deletions(-) diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java index 5430b375302a..34a7aacd4e82 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java @@ -24,8 +24,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.ByteSource; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningScheduledExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.Pair; @@ -56,7 +54,6 @@ public class TierRoutingTaskRunner implements TaskRunner, TaskLogStreamer { private static final Logger LOG = new Logger(TierRoutingTaskRunner.class); private final ConcurrentMap runnerMap = new ConcurrentHashMap<>(); - private final ListeningScheduledExecutorService scheduledExecutorService; final TierRoutingManagementStrategy managementStrategy; @Inject @@ -65,15 +62,10 @@ public TierRoutingTaskRunner( ScheduledExecutorFactory managementExecutorServiceFactory ) { - scheduledExecutorService = MoreExecutors.listeningDecorator(managementExecutorServiceFactory.create( - 1, - "TierRoutingManagement--%d" - )); - managementStrategy = new TierRoutingManagementStrategy(runnerMap, configSupplier, scheduledExecutorService); + managementStrategy = new TierRoutingManagementStrategy(configSupplier, managementExecutorServiceFactory); } - // For Unit Tests - protected ConcurrentMap getRunnerMap() + public ConcurrentMap getRunnerMap() { return runnerMap; } @@ -175,7 +167,7 @@ public Optional getScalingStats() @LifecycleStart public void start() { - managementStrategy.startManagement(null); + managementStrategy.startManagement(this); } @Override @@ -183,7 +175,6 @@ public void start() public void stop() { managementStrategy.stopManagement(); - scheduledExecutorService.shutdown(); } @Override diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java index 1bb323d35b5e..ac2663a12b4a 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java @@ -23,9 +23,16 @@ import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; +import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.logger.Logger; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.TaskRunner; @@ -35,122 +42,160 @@ import javax.annotation.Nullable; import java.util.List; +import java.util.concurrent.CancellationException; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; public class TierRoutingManagementStrategy implements ResourceManagementStrategy { private static final Logger LOG = new Logger(TierRoutingManagementStrategy.class); public static final String ROUTING_TARGET_CONTEXT_KEY = "io.druid.index.tier.target"; public static final String DEFAULT_ROUTE = "__default"; + private final ReadWriteLock startStopStateLock = new ReentrantReadWriteLock(true); private final Supplier configSupplier; - private final ConcurrentMap runnerMap; + private final ScheduledExecutorFactory managementExecutorServiceFactory; private final AtomicBoolean started = new AtomicBoolean(false); - private final ScheduledExecutorService managementExecutorService; private final AtomicLong numberOfUpdates = new AtomicLong(0L); + private final AtomicLong managementEpoch = new AtomicLong(0L); + private volatile TierRoutingTaskRunner runner = null; + private volatile ListeningScheduledExecutorService managementExecutorService = null; public TierRoutingManagementStrategy( - ConcurrentMap runnerMap, Supplier configSupplier, - ScheduledExecutorService managementExecutorService + ScheduledExecutorFactory managementExecutorServiceFactory ) { - this.runnerMap = runnerMap; this.configSupplier = configSupplier; - this.managementExecutorService = managementExecutorService; + this.managementExecutorServiceFactory = managementExecutorServiceFactory; } @Override // State is communicated via configSupplier and runnerMap - public synchronized void startManagement(TierRoutingTaskRunner unused) + public void startManagement(final TierRoutingTaskRunner runner) { - if (!started.compareAndSet(false, true)) { - throw new ISE("Already started"); + try { + startStopStateLock.writeLock().lockInterruptibly(); } - if (managementExecutorService.isShutdown()) { - started.set(false); - throw new ISE("Already stopped"); + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); } - managementExecutorService.scheduleWithFixedDelay( - new Runnable() - { - final AtomicReference priorConfig = new AtomicReference<>(null); - - @Override - public void run() + try { + if (!started.compareAndSet(false, true)) { + throw new ISE("Already started"); + } + this.runner = runner; + managementExecutorService = MoreExecutors.listeningDecorator(managementExecutorServiceFactory.create( + 1, + "TierRoutingManagement--%d" + )); + final ListenableFuture future = managementExecutorService.scheduleWithFixedDelay( + new Runnable() { - try { - // Local management monitors for config changes. - final TierRouteConfig config = configSupplier.get(); - if (config == null) { - throw new ISE("No config found"); - } + final AtomicReference priorConfig = new AtomicReference<>(null); - final TierRouteConfig prior = priorConfig.get(); - if (prior == config) { - LOG.debug("No change in config since last check, skipping update"); - return; + @Override + public void run() + { + try { + startStopStateLock.readLock().lockInterruptibly(); } - - if (!priorConfig.compareAndSet(prior, config)) { - LOG.warn( - "Tier routing config was updated in a racy way... leaving config [%s] and skipping update", - prior - ); - return; + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); } + try { + if (!started.get()) { + LOG.debug("Management not started, returning"); + return; + } + // Local management monitors for config changes. + final TierRouteConfig config = configSupplier.get(); - for (String tier : config.getTiers()) { - if (runnerMap.containsKey(tier)) { - LOG.debug("Tier [%s] already in map", tier); - continue; + final TierRouteConfig prior = priorConfig.get(); + if (prior == config) { + LOG.debug("No change in config since last check, skipping update"); + return; } - final TierTaskRunnerFactory runnerFactory = config.getRouteFactory(tier); - final TaskRunner runner = runnerFactory.build(); - if (runnerMap.putIfAbsent(tier, runner) != null) { - LOG.warn("Tier [%s] lost a race condition, ignoring runner already in map", tier); - continue; + + if (!priorConfig.compareAndSet(prior, config)) { + LOG.warn( + "Tier routing config was updated in a racy way... leaving config [%s] and skipping update", + prior + ); + return; } - try { - synchronized (TierRoutingManagementStrategy.this) { - if (started.get()) { - runner.start(); - } else { - LOG.warn("Tier [%s] trying to start after shutdown", tier); - if (!runnerMap.remove(tier, runner)) { - // This shouldn't happen, but is here as a super safeguard - LOG.warn("Someone else will have to cleanup the runner for tier [%s], they won a race", tier); - } - } + + + final ConcurrentMap runnerMap = runner.getRunnerMap(); + + for (String tier : config.getTiers()) { + if (runnerMap.containsKey(tier)) { + LOG.debug("Tier [%s] already in map", tier); + continue; + } + final TierTaskRunnerFactory runnerFactory = config.getRouteFactory(tier); + final TaskRunner runner = runnerFactory.build(); + if (runnerMap.putIfAbsent(tier, runner) != null) { + LOG.warn("Tier [%s] lost a race condition, ignoring runner already in map", tier); + continue; + } + try { + runner.start(); + } + catch (Exception e) { + LOG.error(e, "Error starting tier [%s], continuing", tier); } } - catch (Exception e) { - LOG.error(e, "Error starting tier [%s], continuing", tier); - } + // TODO: what about tiers that vanish from config? I'm inclined to leave them running in case the vanishing was an error + // Restarting JVM should take care of such a case } - // TODO: what about tiers that vanish from config? I'm inclined to leave them running in case the vanishing was an error - // Restarting JVM should take care of such a case - } - catch (Exception e) { - LOG.error(e, "Tier routing management encountered exception. Trying again"); - } - finally { - // Used in unit tests - synchronized (numberOfUpdates) { - numberOfUpdates.incrementAndGet(); - numberOfUpdates.notifyAll(); + catch (Exception e) { + LOG.error(e, "Tier routing management encountered exception. Trying again"); + } + finally { + startStopStateLock.readLock().unlock(); + // Used in unit tests + synchronized (numberOfUpdates) { + numberOfUpdates.incrementAndGet(); + numberOfUpdates.notifyAll(); + } } } + }, + 0, + 10, // TODO: make this configurable + TimeUnit.SECONDS + ); + + Futures.addCallback(future, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + LOG.info("Success"); + } + + @Override + public void onFailure(Throwable t) + { + if (t instanceof CancellationException) { + LOG.debug("Management thread cancelled"); + } else { + LOG.error(t, "Unhandled exception in management thread for runner %s", runner); } - }, - 0, - 10, // TODO: make this configurable - TimeUnit.SECONDS - ); + } + }); + LOG.info("Started management of %s", runner); + } + finally { + startStopStateLock.writeLock().unlock(); + } } @VisibleForTesting @@ -165,81 +210,125 @@ void waitForUpdate() throws InterruptedException } @Override - public synchronized void stopManagement() + public void stopManagement() { - if (!started.compareAndSet(true, false)) { - LOG.warn("Ignoring repeated stop request"); - return; - } - managementExecutorService.shutdown(); try { - if (!managementExecutorService.awaitTermination(10, TimeUnit.SECONDS)) { - LOG.warn("Could not shut down all tasks. Continuing anyways"); - } + startStopStateLock.writeLock().lockInterruptibly(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.error(e, "Interrupted"); + throw Throwables.propagate(e); } - for (String tier : runnerMap.keySet()) { - final TaskRunner runner = runnerMap.get(tier); - if (runner == null) { - LOG.warn("Race condition for tier [%s]", tier); - continue; + try { + if (!started.compareAndSet(true, false)) { + LOG.warn("Ignoring repeated stop request"); + return; } + managementExecutorService.shutdownNow(); try { - runner.stop(); + if (!managementExecutorService.awaitTermination(10, TimeUnit.SECONDS)) { + LOG.warn("Could not shut down all management tasks! Continuing anyways"); + } + managementExecutorService = null; } - catch (Exception e) { - LOG.error(e, "Error shutting down runner for tier [%s]", tier); + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error(e, "Interrupted"); + } + final ConcurrentMap runnerMap = runner.getRunnerMap(); + + for (String tier : runnerMap.keySet()) { + final TaskRunner runner = runnerMap.remove(tier); + if (runner == null) { + LOG.warn("Race condition for tier [%s]", tier); + continue; + } + try { + runner.stop(); + } + catch (Exception e) { + LOG.error(e, "Error shutting down runner for tier [%s]", tier); + } } - runnerMap.remove(tier); + LOG.info("Stopped management"); + } + finally { + startStopStateLock.writeLock().unlock(); } } @Override - public synchronized ScalingStats getStats() + public ScalingStats getStats() { - final ScalingStats stats = new ScalingStats(0); - final AtomicBoolean foundSomething = new AtomicBoolean(false); - stats.addAllEvents(ImmutableList.copyOf( - FluentIterable - .from(runnerMap.values()) - .transformAndConcat(new Function>() - { - @Nullable - @Override - public List apply(@Nullable TaskRunner runner) + try { + startStopStateLock.readLock().lockInterruptibly(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + try { + if (!started.get()) { + throw new IllegalStateException("Management not started"); + } + final ScalingStats stats = new ScalingStats(0); + final AtomicBoolean foundSomething = new AtomicBoolean(false); + stats.addAllEvents(ImmutableList.copyOf( + FluentIterable + .from(runner.getRunnerMap().values()) + .transformAndConcat(new Function>() { - if (runner == null) { - return ImmutableList.of(); - } - final Optional stats = runner.getScalingStats(); - if (stats.isPresent()) { - foundSomething.set(true); - return stats.get().toList(); - } else { - return ImmutableList.of(); + @Nullable + @Override + public List apply(@Nullable TaskRunner otherRunner) + { + if (otherRunner == null) { + return ImmutableList.of(); + } + final Optional stats = otherRunner.getScalingStats(); + if (stats.isPresent()) { + foundSomething.set(true); + return stats.get().toList(); + } else { + return ImmutableList.of(); + } } - } - }) - )); - return foundSomething.get() ? stats : null; + }) + )); + return foundSomething.get() ? stats : null; + } + finally { + startStopStateLock.readLock().unlock(); + } } public TaskRunner getRunner(Task task) { - final Object tierobj = task.getContextValue(ROUTING_TARGET_CONTEXT_KEY); - final String tier; - if (tierobj == null) { - LOG.debug("No route context found for task [%s]. Using default [%s]", task.getId(), DEFAULT_ROUTE); - tier = DEFAULT_ROUTE; - } else { - tier = tierobj.toString(); + try { + startStopStateLock.readLock().lockInterruptibly(); } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + try { + if (!started.get()) { + throw new IllegalStateException("Management not started"); + } + final Object tierobj = task.getContextValue(ROUTING_TARGET_CONTEXT_KEY); + final String tier; + if (tierobj == null) { + LOG.debug("No route context found for task [%s]. Using default [%s]", task.getId(), DEFAULT_ROUTE); + tier = DEFAULT_ROUTE; + } else { + tier = tierobj.toString(); + } - LOG.info("Using tier [%s] for task [%s]", tier, task.getId()); - - return runnerMap.get(tier); + LOG.info("Using tier [%s] for task [%s]", tier, task.getId()); + return runner.getRunnerMap().get(tier); + } + finally { + startStopStateLock.readLock().unlock(); + } } } diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java index a2fe9338d359..165fc694b39c 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java @@ -27,9 +27,11 @@ import com.google.common.collect.ImmutableSet; import com.metamx.common.IAE; import com.metamx.common.ISE; +import com.metamx.common.concurrent.ScheduledExecutorFactory; import io.druid.concurrent.Execs; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TierRoutingTaskRunner; import io.druid.indexing.overlord.routing.TierRouteConfig; import io.druid.indexing.overlord.routing.TierTaskRunnerFactory; import org.easymock.EasyMock; @@ -39,18 +41,28 @@ import javax.validation.constraints.NotNull; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; public class TierRoutingManagementStrategyTest { private static final String TIER = "test_tier"; - private final ScheduledExecutorService scheduledExecutorService = Execs.scheduledSingleThreaded("routing-test-%s"); - private final TaskRunner runner = EasyMock.createStrictMock(TaskRunner.class); - private final Object memoryBarrier = new Object(); - private final ConcurrentHashMap taskRunnerMap = new ConcurrentHashMap<>(); + private static final ScheduledExecutorFactory scheduledExecutorFactory = new ScheduledExecutorFactory() + { + @Override + public ScheduledExecutorService create(int corePoolSize, String nameFormat) + { + return Executors.newScheduledThreadPool(corePoolSize, Execs.makeThreadFactory(nameFormat)); + } + }; + private final TierRoutingTaskRunner runner = new TierRoutingTaskRunner( + Suppliers.ofInstance(new TierRouteConfig()), + scheduledExecutorFactory + ); + + private final TaskRunner foreignRunner = EasyMock.createStrictMock(TaskRunner.class); private final TierRouteConfig tierRouteConfig = new TierRouteConfig() { @Override @@ -70,9 +82,7 @@ TierTaskRunnerFactory getRouteFactory(@NotNull String tier) @Override public TaskRunner build() { - synchronized (memoryBarrier) { - return runner; - } + return foreignRunner; } }; } else { @@ -82,19 +92,20 @@ public TaskRunner build() }; private TierRoutingManagementStrategy strategy = new TierRoutingManagementStrategy( - taskRunnerMap, Suppliers.ofInstance(tierRouteConfig), - scheduledExecutorService + new ScheduledExecutorFactory() + { + @Override + public ScheduledExecutorService create(int corePoolSize, String nameFormat) + { + return Executors.newScheduledThreadPool(corePoolSize, Execs.makeThreadFactory(nameFormat)); + } + } ); @Test public void testStartManagement() throws Exception { - synchronized (memoryBarrier) { - runner.start(); - EasyMock.expectLastCall().once(); - EasyMock.replay(runner); - } final ExecutorService service = Execs.singleThreaded("TestWatcher"); try { final Future future = service.submit(new Runnable() @@ -111,9 +122,8 @@ public void run() } } }); - strategy.startManagement(null); + strategy.startManagement(runner); future.get(); - EasyMock.verify(runner); } finally { service.shutdownNow(); @@ -123,16 +133,24 @@ public void run() @Test(expected = ISE.class) public void testMultipleStartsFails() throws Exception { - strategy.startManagement(null); - strategy.startManagement(null); + strategy.startManagement(runner); + strategy.startManagement(runner); } - @Test(expected = ISE.class) - public void testMultipleStartsFailsAfterStop() throws Exception + @Test + public void testMultipleStarts() throws Exception { - strategy.startManagement(null); + strategy.startManagement(runner); + strategy.stopManagement(); + strategy.startManagement(runner); + strategy.stopManagement(); + strategy.startManagement(runner); strategy.stopManagement(); - strategy.startManagement(null); + strategy.startManagement(runner); + strategy.stopManagement(); + strategy.startManagement(runner); + strategy.stopManagement(); + strategy.startManagement(runner); } @@ -140,43 +158,27 @@ public void testMultipleStartsFailsAfterStop() throws Exception public void testStopManagement() throws Exception { testStartManagement(); - synchronized (memoryBarrier) { - EasyMock.reset(runner); - runner.stop(); - EasyMock.expectLastCall().once(); - EasyMock.replay(runner); - } strategy.stopManagement(); - Assert.assertTrue(scheduledExecutorService.isShutdown()); - EasyMock.verify(runner); } @Test public void testStopManagementMultiple() throws Exception { testStartManagement(); - synchronized (memoryBarrier) { - EasyMock.reset(runner); - runner.stop(); - EasyMock.expectLastCall().once(); - EasyMock.replay(runner); - } strategy.stopManagement(); strategy.stopManagement(); - EasyMock.verify(runner); } @Test public void testGetEmptyStats() throws Exception { testStartManagement(); - synchronized (memoryBarrier) { - EasyMock.reset(runner); - EasyMock.expect(runner.getScalingStats()).andReturn(Optional.absent()).once(); - EasyMock.replay(runner); - } + EasyMock.reset(foreignRunner); + EasyMock.expect(foreignRunner.getScalingStats()).andReturn(Optional.absent()).once(); + EasyMock.replay(foreignRunner); Assert.assertNull(strategy.getStats()); - EasyMock.verify(runner); + EasyMock.verify(foreignRunner); + EasyMock.reset(foreignRunner); } @Test @@ -184,26 +186,28 @@ public void testSimpleOneStates() throws Exception { testStartManagement(); final ScalingStats.ScalingEvent event = EasyMock.createStrictMock(ScalingStats.ScalingEvent.class); - synchronized (memoryBarrier) { + synchronized (this) { final ScalingStats stats = new ScalingStats(0); stats.addAllEvents(ImmutableList.of(event)); - EasyMock.reset(runner); - EasyMock.expect(runner.getScalingStats()).andReturn(Optional.of(stats)).once(); - EasyMock.replay(runner); + EasyMock.reset(foreignRunner); + EasyMock.expect(foreignRunner.getScalingStats()).andReturn(Optional.of(stats)).once(); + EasyMock.replay(foreignRunner); } final ScalingStats stats = strategy.getStats(); - EasyMock.verify(runner); Assert.assertFalse(stats.toList().isEmpty()); Assert.assertEquals(event, stats.toList().get(0)); + EasyMock.verify(foreignRunner); + // Reset for teardown + EasyMock.reset(foreignRunner); } @Test public void testGetRunner() throws Exception { - final TaskRunner defaultRunner = EasyMock.createStrictMock(TaskRunner.class); - taskRunnerMap.put(TierRoutingManagementStrategy.DEFAULT_ROUTE, defaultRunner); testStartManagement(); - Assert.assertEquals(runner, strategy.getRunner( + final TaskRunner defaultRunner = EasyMock.createStrictMock(TaskRunner.class); + runner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, defaultRunner); + Assert.assertEquals(foreignRunner, strategy.getRunner( new NoopTask("task_id", 0, 0, "YES", null, ImmutableMap.of( TierRoutingManagementStrategy.ROUTING_TARGET_CONTEXT_KEY, TIER @@ -221,8 +225,6 @@ public void testGetRunner() throws Exception @After public void tearDown() { - if (!scheduledExecutorService.isShutdown()) { - scheduledExecutorService.shutdownNow(); - } + strategy.stopManagement(); } } \ No newline at end of file From 072a8b3c8288eacdc93b5cacfe4b407de036f443 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 24 Jun 2016 12:26:34 -0700 Subject: [PATCH 14/16] Fix management strategy threading. --- .../TierRoutingManagementStrategy.java | 16 ++- .../TierRoutingManagementStrategyTest.java | 122 +++++++++++++++++- 2 files changed, 132 insertions(+), 6 deletions(-) diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java index ac2663a12b4a..ccf426d431e5 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java @@ -89,10 +89,11 @@ public void startManagement(final TierRoutingTaskRunner runner) if (!started.compareAndSet(false, true)) { throw new ISE("Already started"); } + final long managementEpoch = this.managementEpoch.getAndIncrement(); this.runner = runner; managementExecutorService = MoreExecutors.listeningDecorator(managementExecutorServiceFactory.create( 1, - "TierRoutingManagement--%d" + String.format("TierRoutingManagement-%d-%%d", managementEpoch) )); final ListenableFuture future = managementExecutorService.scheduleWithFixedDelay( new Runnable() @@ -114,6 +115,15 @@ public void run() LOG.debug("Management not started, returning"); return; } + final long actualEpoch = TierRoutingManagementStrategy.this.managementEpoch.get(); + if (!(managementEpoch == actualEpoch)) { + LOG.info( + "Management epoch [%d] is stale. Current epoch is [%d]. Skipping management for this thread", + managementEpoch, + actualEpoch + ); + return; + } // Local management monitors for config changes. final TierRouteConfig config = configSupplier.get(); @@ -269,7 +279,7 @@ public ScalingStats getStats() } try { if (!started.get()) { - throw new IllegalStateException("Management not started"); + throw new ISE("Management not started"); } final ScalingStats stats = new ScalingStats(0); final AtomicBoolean foundSomething = new AtomicBoolean(false); @@ -313,7 +323,7 @@ public TaskRunner getRunner(Task task) } try { if (!started.get()) { - throw new IllegalStateException("Management not started"); + throw new ISE("Management not started"); } final Object tierobj = task.getContextValue(ROUTING_TARGET_CONTEXT_KEY); final String tier; diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java index 165fc694b39c..f0f7964739bb 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java @@ -25,6 +25,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; @@ -40,16 +44,22 @@ import org.junit.Test; import javax.validation.constraints.NotNull; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import java.util.Set; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; public class TierRoutingManagementStrategyTest { private static final String TIER = "test_tier"; - private static final ScheduledExecutorFactory scheduledExecutorFactory = new ScheduledExecutorFactory() + private static final ScheduledExecutorFactory SCHEDULED_EXECUTOR_FACTORY = new ScheduledExecutorFactory() { @Override public ScheduledExecutorService create(int corePoolSize, String nameFormat) @@ -59,7 +69,7 @@ public ScheduledExecutorService create(int corePoolSize, String nameFormat) }; private final TierRoutingTaskRunner runner = new TierRoutingTaskRunner( Suppliers.ofInstance(new TierRouteConfig()), - scheduledExecutorFactory + SCHEDULED_EXECUTOR_FACTORY ); private final TaskRunner foreignRunner = EasyMock.createStrictMock(TaskRunner.class); @@ -91,7 +101,7 @@ public TaskRunner build() } }; - private TierRoutingManagementStrategy strategy = new TierRoutingManagementStrategy( + private final TierRoutingManagementStrategy strategy = new TierRoutingManagementStrategy( Suppliers.ofInstance(tierRouteConfig), new ScheduledExecutorFactory() { @@ -176,11 +186,19 @@ public void testGetEmptyStats() throws Exception EasyMock.reset(foreignRunner); EasyMock.expect(foreignRunner.getScalingStats()).andReturn(Optional.absent()).once(); EasyMock.replay(foreignRunner); + runner.getRunnerMap().put("some_tier", foreignRunner); Assert.assertNull(strategy.getStats()); EasyMock.verify(foreignRunner); EasyMock.reset(foreignRunner); } + @Test + public void testGetNoStats() throws Exception + { + testStartManagement(); + Assert.assertNull(strategy.getStats()); + } + @Test public void testSimpleOneStates() throws Exception { @@ -193,6 +211,7 @@ public void testSimpleOneStates() throws Exception EasyMock.expect(foreignRunner.getScalingStats()).andReturn(Optional.of(stats)).once(); EasyMock.replay(foreignRunner); } + runner.getRunnerMap().put("some_tier", foreignRunner); final ScalingStats stats = strategy.getStats(); Assert.assertFalse(stats.toList().isEmpty()); Assert.assertEquals(event, stats.toList().get(0)); @@ -207,6 +226,7 @@ public void testGetRunner() throws Exception testStartManagement(); final TaskRunner defaultRunner = EasyMock.createStrictMock(TaskRunner.class); runner.getRunnerMap().put(TierRoutingManagementStrategy.DEFAULT_ROUTE, defaultRunner); + runner.getRunnerMap().put(TIER, foreignRunner); Assert.assertEquals(foreignRunner, strategy.getRunner( new NoopTask("task_id", 0, 0, "YES", null, ImmutableMap.of( TierRoutingManagementStrategy.ROUTING_TARGET_CONTEXT_KEY, @@ -222,6 +242,102 @@ public void testGetRunner() throws Exception )); } + @Test + public void testConcurrency() throws Exception + { + final int numTests = 100; + final ListeningExecutorService executorService = MoreExecutors.listeningDecorator(Execs.multiThreaded( + numTests, + "test-hammer-%d" + )); + final ArrayList> futures = new ArrayList<>(numTests); + final CyclicBarrier barrier = new CyclicBarrier(numTests); + final Random random = new Random(374898704198L); + final List tasks = ImmutableList.of( + new Runnable() + { + @Override + public void run() + { + try { + strategy.startManagement(runner); + } + catch (ISE e) { + if (!"Already started".equals(e.getMessage())) { + throw e; + } + } + } + }, + new Runnable() + { + @Override + public void run() + { + strategy.stopManagement(); + } + }, + new Runnable() + { + @Override + public void run() + { + try { + strategy.getStats(); + } + catch (ISE e) { + if (!"Management not started".equals(e.getMessage())) { + throw e; + } + } + } + }, + new Runnable() + { + @Override + public void run() + { + try { + strategy.getRunner(new NoopTask("task_id", 0, 0, "YES", null, ImmutableMap.of( + TierRoutingManagementStrategy.ROUTING_TARGET_CONTEXT_KEY, + TIER + ))); + } + catch (ISE e) { + if (!"Management not started".equals(e.getMessage())) { + throw e; + } + } + } + } + ); + for (int i = 0; i < numTests; ++i) { + final Runnable task = tasks.get(random.nextInt(tasks.size())); + futures.add(executorService.submit(new Runnable() + { + @Override + public void run() + { + try { + barrier.await(); + } + catch (InterruptedException | BrokenBarrierException e) { + throw Throwables.propagate(e); + } + for (int j = 0; j < 1000; ++j) { + task.run(); + } + } + })); + } + try { + Futures.allAsList(futures).get(1, TimeUnit.MINUTES); + } + finally { + executorService.shutdownNow(); + } + } + @After public void tearDown() { From e99ed4ddb79b256fbd075370c41b9060e181562a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 4 Nov 2016 08:22:17 -0700 Subject: [PATCH 15/16] Basic fixes for merging with master --- extensions-core/tasktier/pom.xml | 2 +- .../src/main/java/io/druid/cli/CliTierFork.java | 2 +- .../src/main/java/io/druid/cli/CliTierLocal.java | 2 +- .../java/io/druid/cli/ExecutorLifecycleProvider.java | 2 +- .../src/main/java/io/druid/cli/ForkAnnouncer.java | 4 ++-- .../overlord/AbstractTierRemoteTaskRunner.java | 12 ++++++------ .../java/io/druid/indexing/overlord/PortWriter.java | 10 +++++----- .../overlord/PostingTierRemoteTaskRunner.java | 4 ++-- .../druid/indexing/overlord/TierLocalTaskRunner.java | 12 ++++++------ .../indexing/overlord/TierRoutingTaskRunner.java | 12 ++++++------ .../overlord/TierRoutingTaskRunnerFactory.java | 4 +--- .../druid/indexing/overlord/TierTaskDiscovery.java | 2 +- .../autoscaling/TierRoutingManagementStrategy.java | 6 +++--- .../indexing/overlord/config/TierConfigModule.java | 2 +- .../indexing/overlord/resources/DeadhandMonitor.java | 8 ++++---- .../overlord/resources/DeadhandResource.java | 3 +-- .../indexing/overlord/resources/TaskLogResource.java | 2 +- .../overlord/routing/DynamicUpstreamReporter.java | 2 +- .../routing/RemoteTaskRunnerTierFactory.java | 2 +- .../routing/TaskStatusPostToLeaderReporter.java | 4 ++-- .../overlord/routing/UnknownRouteFactory.java | 2 +- .../test/java/io/druid/cli/CliTierCreatorTest.java | 3 ++- .../src/test/java/io/druid/cli/CliTierForkTest.java | 3 ++- .../io/druid/cli/ExecutorLifecycleProviderTest.java | 3 ++- .../test/java/io/druid/cli/ForkAnnouncerTest.java | 3 ++- .../overlord/AbstractTierRemoteTaskRunnerTest.java | 6 +++--- .../indexing/overlord/TierRoutingTaskRunnerTest.java | 4 ++-- .../TierRoutingManagementStrategyTest.java | 9 +++++---- .../overlord/config/TierConfigResourceTest.java | 3 ++- .../overlord/resources/DeadhandResourceTest.java | 3 ++- .../overlord/resources/TaskLogResourceTest.java | 3 ++- .../routing/RemoteTaskRunnerTierFactoryTest.java | 2 +- .../overlord/routing/TierRouteConfigTest.java | 3 ++- .../overlord/routing/UnknownRouteFactoryTest.java | 3 ++- pom.xml | 11 ----------- 35 files changed, 77 insertions(+), 81 deletions(-) diff --git a/extensions-core/tasktier/pom.xml b/extensions-core/tasktier/pom.xml index 622a5b4385d1..af02d9d2769f 100644 --- a/extensions-core/tasktier/pom.xml +++ b/extensions-core/tasktier/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.9.2-SNAPSHOT + 0.9.3-SNAPSHOT ../../pom.xml diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java index 365187152331..5cbff6d5cd42 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierFork.java @@ -27,7 +27,6 @@ import com.google.inject.Module; import com.google.inject.Provider; import com.google.inject.Provides; -import com.metamx.common.logger.Logger; import io.airlift.airline.Command; import io.druid.guice.Jerseys; import io.druid.guice.LazySingleton; @@ -49,6 +48,7 @@ import io.druid.indexing.overlord.resources.TierRunningCheckResource; import io.druid.indexing.worker.executor.ExecutorLifecycle; import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; +import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; import io.druid.server.initialization.jetty.ChatHandlerServerModule; diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierLocal.java b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierLocal.java index d378a041e539..a94e29ca3978 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierLocal.java +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/CliTierLocal.java @@ -25,7 +25,6 @@ import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.util.Modules; -import com.metamx.common.logger.Logger; import io.airlift.airline.Command; import io.druid.guice.LazySingleton; import io.druid.guice.ManageLifecycle; @@ -34,6 +33,7 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.worker.Worker; import io.druid.indexing.worker.WorkerCuratorCoordinator; +import io.druid.java.util.common.logger.Logger; import io.druid.server.initialization.IndexerZkConfig; import org.apache.curator.framework.CuratorFramework; diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java b/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java index 1f46446220c5..455db8643825 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java @@ -23,7 +23,6 @@ import com.google.common.base.Throwables; import com.google.inject.Inject; import com.google.inject.Provider; -import com.metamx.common.logger.Logger; import io.druid.guice.annotations.Json; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; @@ -31,6 +30,7 @@ import io.druid.indexing.overlord.TierLocalTaskRunner; import io.druid.indexing.worker.executor.ExecutorLifecycle; import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; +import io.druid.java.util.common.logger.Logger; import java.io.File; import java.io.IOException; diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/ForkAnnouncer.java b/extensions-core/tasktier/src/main/java/io/druid/cli/ForkAnnouncer.java index 27b903029ca3..97195564be42 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/cli/ForkAnnouncer.java +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/ForkAnnouncer.java @@ -22,13 +22,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.inject.Inject; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; import io.druid.curator.announcement.Announcer; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Self; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.config.TierForkZkConfig; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.server.DruidNode; import java.io.IOException; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunner.java index ebd022d5e430..b65634363997 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunner.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunner.java @@ -34,12 +34,6 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; -import com.metamx.common.ISE; -import com.metamx.common.Pair; -import com.metamx.common.RetryUtils; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; @@ -55,6 +49,12 @@ import io.druid.indexing.overlord.resources.TaskLogResource; import io.druid.indexing.overlord.resources.TierRunningCheckResource; import io.druid.indexing.overlord.routing.TaskStatusReporter; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.RetryUtils; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.server.DruidNode; import io.druid.tasklogs.TaskLogStreamer; import org.jboss.netty.handler.codec.http.HttpChunk; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PortWriter.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PortWriter.java index 529c163f3a7e..adb782bccdc7 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PortWriter.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PortWriter.java @@ -21,12 +21,12 @@ import com.google.common.base.Throwables; import com.google.inject.Inject; -import com.metamx.common.ISE; -import com.metamx.common.StringUtils; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; import io.druid.guice.annotations.Self; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import java.io.File; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PostingTierRemoteTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PostingTierRemoteTaskRunner.java index 1007aba8534f..1858f880ad02 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PostingTierRemoteTaskRunner.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/PostingTierRemoteTaskRunner.java @@ -5,11 +5,11 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.logger.Logger; import com.metamx.http.client.HttpClient; import io.druid.guice.annotations.Global; import io.druid.indexing.common.task.Task; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import javax.annotation.Nullable; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java index d122eb548915..776d9f97ce46 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java @@ -37,12 +37,6 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; -import com.metamx.common.ISE; -import com.metamx.common.Pair; -import com.metamx.common.RE; -import com.metamx.common.StringUtils; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; @@ -64,6 +58,12 @@ import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; import io.druid.indexing.overlord.resources.DeadhandResource; import io.druid.indexing.worker.config.WorkerConfig; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.RE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.server.DruidNode; import io.druid.tasklogs.TaskLogPusher; import io.druid.tasklogs.TaskLogStreamer; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java index 34a7aacd4e82..0cdb9f97a820 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunner.java @@ -25,17 +25,17 @@ import com.google.common.io.ByteSource; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; -import com.metamx.common.IAE; -import com.metamx.common.Pair; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.autoscaling.ScalingStats; import io.druid.indexing.overlord.autoscaling.TierRoutingManagementStrategy; import io.druid.indexing.overlord.routing.TierRouteConfig; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.common.logger.Logger; import io.druid.tasklogs.TaskLogStreamer; import java.io.IOException; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java index 4e3441452734..236057a67b1b 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierRoutingTaskRunnerFactory.java @@ -19,12 +19,10 @@ package io.druid.indexing.overlord; -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Supplier; import com.google.inject.Inject; -import com.metamx.common.concurrent.ScheduledExecutorFactory; import io.druid.indexing.overlord.routing.TierRouteConfig; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; public class TierRoutingTaskRunnerFactory implements TaskRunnerFactory { diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierTaskDiscovery.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierTaskDiscovery.java index a44816836fa5..3fe0f64383ee 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierTaskDiscovery.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierTaskDiscovery.java @@ -25,9 +25,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; -import com.metamx.common.logger.Logger; import io.druid.guice.annotations.Json; import io.druid.indexing.overlord.config.TierForkZkConfig; +import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import org.apache.curator.framework.CuratorFramework; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java index ccf426d431e5..cc42bcb739a2 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java @@ -31,14 +31,14 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.logger.Logger; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.overlord.TierRoutingTaskRunner; import io.druid.indexing.overlord.routing.TierRouteConfig; import io.druid.indexing.overlord.routing.TierTaskRunnerFactory; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import io.druid.java.util.common.logger.Logger; import javax.annotation.Nullable; import java.util.List; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java index eca0760557e6..f8f2a90223d3 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/config/TierConfigModule.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Inject; -import com.metamx.common.logger.Logger; import io.druid.audit.AuditInfo; import io.druid.audit.AuditManager; import io.druid.common.config.JacksonConfigManager; @@ -37,6 +36,7 @@ import io.druid.indexing.overlord.routing.TierRouteConfig; import io.druid.indexing.overlord.routing.TierTaskRunnerFactory; import io.druid.initialization.DruidModule; +import io.druid.java.util.common.logger.Logger; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandMonitor.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandMonitor.java index 52c44190d050..d161c2578855 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandMonitor.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandMonitor.java @@ -26,13 +26,13 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; -import com.metamx.common.ISE; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; import io.druid.concurrent.Execs; import io.druid.indexing.overlord.TierLocalTaskRunner; import io.druid.indexing.overlord.config.TierLocalTaskRunnerConfig; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.common.logger.Logger; import javax.annotation.Nullable; import java.io.File; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandResource.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandResource.java index 82664b8b4de6..fff59310ccaa 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandResource.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/DeadhandResource.java @@ -19,8 +19,7 @@ package io.druid.indexing.overlord.resources; - -import com.metamx.common.logger.Logger; +import io.druid.java.util.common.logger.Logger; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.POST; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TaskLogResource.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TaskLogResource.java index 7a05a785a729..0ac3824431ff 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TaskLogResource.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/resources/TaskLogResource.java @@ -20,9 +20,9 @@ package io.druid.indexing.overlord.resources; import com.google.common.base.Preconditions; -import com.metamx.common.logger.Logger; import io.druid.indexing.common.tasklogs.LogUtils; import io.druid.indexing.overlord.TierLocalTaskRunner; +import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import javax.ws.rs.DefaultValue; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporter.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporter.java index 68fb0c79803a..b819849beba8 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporter.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/DynamicUpstreamReporter.java @@ -22,12 +22,12 @@ import com.google.common.base.Optional; import com.google.inject.Inject; import com.google.inject.name.Named; -import com.metamx.common.logger.Logger; import com.metamx.http.client.HttpClient; import io.druid.guice.annotations.Global; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.overlord.TaskMaster; import io.druid.indexing.overlord.TaskRunner; +import io.druid.java.util.common.logger.Logger; import org.apache.curator.x.discovery.ServiceDiscovery; public class DynamicUpstreamReporter implements TaskStatusReporter diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactory.java index 935e81188119..24730ce7e60f 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactory.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactory.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; -import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.http.client.HttpClient; import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Json; @@ -36,6 +35,7 @@ import io.druid.indexing.overlord.autoscaling.ResourceManagementSchedulerConfig; import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import io.druid.server.initialization.IndexerZkConfig; import org.apache.curator.framework.CuratorFramework; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java index 4614738420c0..3737dcce8e14 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/TaskStatusPostToLeaderReporter.java @@ -24,14 +24,14 @@ import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.google.inject.name.Named; -import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; import com.metamx.http.client.response.StatusResponseHandler; import com.metamx.http.client.response.StatusResponseHolder; import io.druid.guice.annotations.Global; import io.druid.indexing.common.TaskStatus; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceInstance; diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/UnknownRouteFactory.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/UnknownRouteFactory.java index d63b35c07944..78e03c735a82 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/UnknownRouteFactory.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/routing/UnknownRouteFactory.java @@ -19,8 +19,8 @@ package io.druid.indexing.overlord.routing; -import com.metamx.common.UOE; import io.druid.indexing.overlord.TaskRunner; +import io.druid.java.util.common.UOE; public class UnknownRouteFactory implements TierTaskRunnerFactory { diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierCreatorTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierCreatorTest.java index 67f9b268207f..c5a7e1af0e98 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierCreatorTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierCreatorTest.java @@ -52,4 +52,5 @@ public String apply(@Nullable CommandGroupMetadata input) } )); } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java index 456d8da15dac..84dd61ded89f 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/CliTierForkTest.java @@ -150,4 +150,5 @@ public void tearDown() // Clear interrupt flag Thread.interrupted(); } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/ExecutorLifecycleProviderTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/ExecutorLifecycleProviderTest.java index 801bd29c8d0b..4fe22dce98df 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/cli/ExecutorLifecycleProviderTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/ExecutorLifecycleProviderTest.java @@ -226,4 +226,5 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.fromCode(getId(), status); } } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/cli/ForkAnnouncerTest.java b/extensions-core/tasktier/src/test/java/io/druid/cli/ForkAnnouncerTest.java index c5e2ff2aff0d..209227f43771 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/cli/ForkAnnouncerTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/cli/ForkAnnouncerTest.java @@ -74,4 +74,5 @@ public void testUnannounceTask() throws Exception forkAnnouncer.unannounceTask(); EasyMock.verify(announcer); } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunnerTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunnerTest.java index e5301f5232ad..2311c27416b4 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunnerTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/AbstractTierRemoteTaskRunnerTest.java @@ -39,9 +39,6 @@ import com.google.inject.Injector; import com.google.inject.Module; import com.google.inject.util.Modules; -import com.metamx.common.ISE; -import com.metamx.common.StringUtils; -import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; @@ -58,6 +55,9 @@ import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.resources.TierRunningCheckResource; import io.druid.initialization.Initialization; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import io.druid.server.DruidNode; import org.easymock.Capture; import org.easymock.EasyMock; diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierRoutingTaskRunnerTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierRoutingTaskRunnerTest.java index 59c44c113895..74a0f76df6ba 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierRoutingTaskRunnerTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/TierRoutingTaskRunnerTest.java @@ -24,8 +24,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.ByteSource; import com.google.common.util.concurrent.SettableFuture; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; import io.druid.concurrent.Execs; import io.druid.indexing.common.TaskLocation; import io.druid.indexing.common.TaskStatus; @@ -34,6 +32,8 @@ import io.druid.indexing.overlord.autoscaling.ScalingStats; import io.druid.indexing.overlord.autoscaling.TierRoutingManagementStrategy; import io.druid.indexing.overlord.routing.TierRouteConfig; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import io.druid.tasklogs.TaskLogStreamer; import org.easymock.EasyMock; import org.junit.After; diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java index f0f7964739bb..9287b375a732 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategyTest.java @@ -29,15 +29,15 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.metamx.common.IAE; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; import io.druid.concurrent.Execs; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.overlord.TierRoutingTaskRunner; import io.druid.indexing.overlord.routing.TierRouteConfig; import io.druid.indexing.overlord.routing.TierTaskRunnerFactory; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -343,4 +343,5 @@ public void tearDown() { strategy.stopManagement(); } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/config/TierConfigResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/config/TierConfigResourceTest.java index 3f5ef3e1a192..a125f7dd5d27 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/config/TierConfigResourceTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/config/TierConfigResourceTest.java @@ -123,4 +123,5 @@ private HttpServletRequest prepareRequest() EasyMock.expect(request.getRemoteHost()).andReturn("localhost").anyTimes(); return request; } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandResourceTest.java index e242b87b0dc4..d5264c7c43c5 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandResourceTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/DeadhandResourceTest.java @@ -300,4 +300,5 @@ public void close() throws IOException } }; } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java index fae7fd511d88..6de7feba2b0b 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/resources/TaskLogResourceTest.java @@ -150,4 +150,5 @@ public void testBuildURLBadOffset() throws Exception final DruidNode testNode = new DruidNode("testService", "somehost", 1234); TaskLogResource.buildURL(testNode, -1); } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java index 485f4736480d..d3f6a46c044d 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/RemoteTaskRunnerTierFactoryTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; -import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.http.client.HttpClient; import io.druid.concurrent.Execs; import io.druid.indexing.overlord.RemoteTaskRunner; @@ -32,6 +31,7 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TierRouteConfigTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TierRouteConfigTest.java index 8e4a4dd983bb..9376a69fde3a 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TierRouteConfigTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/TierRouteConfigTest.java @@ -87,4 +87,5 @@ private void populateConfig(final String tier, final TierTaskRunnerFactory facto ) ); } -} \ No newline at end of file +} + diff --git a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/UnknownRouteFactoryTest.java b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/UnknownRouteFactoryTest.java index ba733ff821e4..5bdf5327b56b 100644 --- a/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/UnknownRouteFactoryTest.java +++ b/extensions-core/tasktier/src/test/java/io/druid/indexing/overlord/routing/UnknownRouteFactoryTest.java @@ -30,4 +30,5 @@ public void testBuild() throws Exception { new UnknownRouteFactory().build(); } -} \ No newline at end of file +} + diff --git a/pom.xml b/pom.xml index 30c65d380d3f..82022d12e3a4 100644 --- a/pom.xml +++ b/pom.xml @@ -164,17 +164,6 @@ netty 3.9.5.Final - - com.metamx - java-util - ${metamx.java-util.version} - - - org.slf4j - slf4j-api - - - com.metamx bytebuffer-collections From bf21e92fd1563c1c439e5c8749c0f2f6c4750544 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 14 Nov 2016 08:37:14 -0800 Subject: [PATCH 16/16] checkpoint --- .../main/java/io/druid/cli/ExecutorLifecycleProvider.java | 5 ++--- .../io/druid/indexing/overlord/TierLocalTaskRunner.java | 3 ++- .../overlord/autoscaling/TierRoutingManagementStrategy.java | 6 ++++-- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java b/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java index 455db8643825..6c025830c9e3 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java +++ b/extensions-core/tasktier/src/main/java/io/druid/cli/ExecutorLifecycleProvider.java @@ -85,9 +85,8 @@ public InputStream getParentStream() { return parentStream; } - } - .setStatusFile(statusFile) - .setTaskFile(taskFile), + }.setStatusFile(statusFile) + .setTaskFile(taskFile), taskConfig, taskActionClientFactory, taskRunner, diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java index 776d9f97ce46..845a1bd3472e 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/TierLocalTaskRunner.java @@ -112,7 +112,7 @@ /** * The TierLocalTaskRunner is a gateway between general cluster task information and forked stuff locally on a - * single machine. It communicate existence via PORT_FILE_NAME in the task attempt directory. + * single machine. It communicates existence via PORT_FILE_NAME in the task attempt directory. * Using the PORT_FILE_NAME the TierLocalTaskRunner can identify peons on the local machine. * The TierLocalTaskRunner is able to restart without affecting the state of the peons. * The Peons run some special magic in io.druid.indexing.overlord.TierModule to expose an ability to shutdown the VMs @@ -652,6 +652,7 @@ public void run() port, DeadhandResource.DEADHAND_PATH ); + log.trace("Starting deadhand probe to [%s]", url); httpClient.go( new Request(HttpMethod.POST, url), new HttpResponseHandler() diff --git a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java index cc42bcb739a2..5c0dcfc634b7 100644 --- a/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java +++ b/extensions-core/tasktier/src/main/java/io/druid/indexing/overlord/autoscaling/TierRoutingManagementStrategy.java @@ -214,7 +214,9 @@ void waitForUpdate() throws InterruptedException final long startingUpdates = numberOfUpdates.get(); while (startingUpdates == numberOfUpdates.get()) { synchronized (numberOfUpdates) { - numberOfUpdates.wait(); + if(startingUpdates == numberOfUpdates.get()) { + numberOfUpdates.wait(); + } } } } @@ -243,7 +245,7 @@ public void stopManagement() } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.error(e, "Interrupted"); + throw Throwables.propagate(e); } final ConcurrentMap runnerMap = runner.getRunnerMap();