Skip to content

Commit

Permalink
add pending task based resource management strategy (#2086)
Browse files Browse the repository at this point in the history
  • Loading branch information
nishantmonu51 authored and xvrl committed Apr 27, 2016
1 parent bf5e5e7 commit c29cb7d
Show file tree
Hide file tree
Showing 23 changed files with 1,437 additions and 205 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -382,6 +382,17 @@ public Collection<RemoteTaskRunnerWorkItem> getPendingTasks()
return ImmutableList.copyOf(pendingTasks.values());
}

public Collection<Task> getPendingTaskPayloads()
{
// return a snapshot of current pending task payloads.
return ImmutableList.copyOf(pendingTaskPayloads.values());
}

public RemoteTaskRunnerConfig getConfig()
{
return config;
}

@Override
public Collection<RemoteTaskRunnerWorkItem> getKnownTasks()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@
import io.druid.indexing.overlord.autoscaling.NoopResourceManagementStrategy;
import io.druid.indexing.overlord.autoscaling.ResourceManagementSchedulerConfig;
import io.druid.indexing.overlord.autoscaling.ResourceManagementStrategy;
import io.druid.indexing.overlord.autoscaling.SimpleResourceManagementConfig;
import io.druid.indexing.overlord.autoscaling.SimpleResourceManagementStrategy;
import io.druid.indexing.overlord.autoscaling.SimpleWorkerResourceManagementConfig;
import io.druid.indexing.overlord.autoscaling.SimpleWorkerResourceManagementStrategy;
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import io.druid.server.initialization.IndexerZkConfig;
Expand All @@ -44,15 +44,14 @@
public class RemoteTaskRunnerFactory implements TaskRunnerFactory<RemoteTaskRunner>
{
public static final String TYPE_NAME = "remote";
private static final Logger LOG = new Logger(RemoteTaskRunnerFactory.class);
private final CuratorFramework curator;
private final RemoteTaskRunnerConfig remoteTaskRunnerConfig;
private final IndexerZkConfig zkPaths;
private final ObjectMapper jsonMapper;
private final HttpClient httpClient;
private final Supplier<WorkerBehaviorConfig> workerConfigRef;
private final SimpleResourceManagementConfig config;
private final ResourceManagementSchedulerConfig resourceManagementSchedulerConfig;
private final ResourceManagementStrategy resourceManagementStrategy;
private final ScheduledExecutorFactory factory;

@Inject
Expand All @@ -64,8 +63,8 @@ public RemoteTaskRunnerFactory(
@Global final HttpClient httpClient,
final Supplier<WorkerBehaviorConfig> workerConfigRef,
final ScheduledExecutorFactory factory,
final SimpleResourceManagementConfig config,
final ResourceManagementSchedulerConfig resourceManagementSchedulerConfig
final ResourceManagementSchedulerConfig resourceManagementSchedulerConfig,
final ResourceManagementStrategy resourceManagementStrategy
)
{
this.curator = curator;
Expand All @@ -74,25 +73,14 @@ public RemoteTaskRunnerFactory(
this.jsonMapper = jsonMapper;
this.httpClient = httpClient;
this.workerConfigRef = workerConfigRef;
this.config = config;
this.resourceManagementSchedulerConfig = resourceManagementSchedulerConfig;
this.resourceManagementStrategy = resourceManagementStrategy;
this.factory = factory;
}

@Override
public RemoteTaskRunner build()
{
final ResourceManagementStrategy<WorkerTaskRunner> resourceManagementStrategy;
if (resourceManagementSchedulerConfig.isDoAutoscale()) {
resourceManagementStrategy = new SimpleResourceManagementStrategy(
config,
workerConfigRef,
resourceManagementSchedulerConfig,
factory.create(1, "RemoteTaskRunner-ResourceManagement--%d")
);
} else {
resourceManagementStrategy = new NoopResourceManagementStrategy<>();
}
return new RemoteTaskRunner(
jsonMapper,
remoteTaskRunnerConfig,
Expand All @@ -105,7 +93,9 @@ public RemoteTaskRunner build()
httpClient,
workerConfigRef,
factory.create(1, "RemoteTaskRunner-Scheduled-Cleanup--%d"),
resourceManagementStrategy
resourceManagementSchedulerConfig.isDoAutoscale()
? resourceManagementStrategy
: new NoopResourceManagementStrategy<>()
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
package io.druid.indexing.overlord;

import com.google.common.base.Predicate;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.config.WorkerTaskRunnerConfig;
import io.druid.indexing.worker.Worker;

import java.util.Collection;
Expand All @@ -44,4 +46,9 @@ public interface WorkerTaskRunner extends TaskRunner
* @return
*/
Collection<Worker> markWorkersLazy(Predicate<ImmutableWorkerInfo> isLazyWorker, int maxWorkers);

WorkerTaskRunnerConfig getConfig();

Collection<Task> getPendingTaskPayloads();

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* 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.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.emitter.EmittingLogger;
import io.druid.granularity.PeriodGranularity;
import io.druid.indexing.overlord.WorkerTaskRunner;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Period;

import java.util.concurrent.ScheduledExecutorService;

/**
*/
public abstract class AbstractWorkerResourceManagementStrategy implements ResourceManagementStrategy<WorkerTaskRunner>
{
private static final EmittingLogger log = new EmittingLogger(AbstractWorkerResourceManagementStrategy.class);

private final ResourceManagementSchedulerConfig resourceManagementSchedulerConfig;
private final ScheduledExecutorService exec;
private final Object lock = new Object();

private volatile boolean started = false;

protected AbstractWorkerResourceManagementStrategy(
ResourceManagementSchedulerConfig resourceManagementSchedulerConfig,
ScheduledExecutorService exec
)
{
this.resourceManagementSchedulerConfig = resourceManagementSchedulerConfig;
this.exec = exec;
}

@Override
public void startManagement(final WorkerTaskRunner runner)
{
synchronized (lock) {
if (started) {
return;
}

log.info("Started Resource Management Scheduler");

ScheduledExecutors.scheduleAtFixedRate(
exec,
resourceManagementSchedulerConfig.getProvisionPeriod().toStandardDuration(),
new Runnable()
{
@Override
public void run()
{
// Any Errors are caught by ScheduledExecutors
doProvision(runner);
}
}
);

// Schedule termination of worker nodes periodically
Period period = resourceManagementSchedulerConfig.getTerminatePeriod();
PeriodGranularity granularity = new PeriodGranularity(
period,
resourceManagementSchedulerConfig.getOriginTime(),
null
);
final long startTime = granularity.next(granularity.truncate(new DateTime().getMillis()));

ScheduledExecutors.scheduleAtFixedRate(
exec,
new Duration(System.currentTimeMillis(), startTime),
resourceManagementSchedulerConfig.getTerminatePeriod().toStandardDuration(),
new Runnable()
{
@Override
public void run()
{
// Any Errors are caught by ScheduledExecutors
doTerminate(runner);
}
}
);

started = true;

}
}

abstract boolean doTerminate(WorkerTaskRunner runner);

abstract boolean doProvision(WorkerTaskRunner runner);

@Override
public void stopManagement()
{
synchronized (lock) {
if (!started) {
return;
}
log.info("Stopping Resource Management Scheduler");
exec.shutdown();
started = false;
}
}

}
Original file line number Diff line number Diff line change
@@ -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.autoscaling;

import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Period;

/**
*/
public class PendingTaskBasedWorkerResourceManagementConfig extends SimpleWorkerResourceManagementConfig
{
@JsonProperty
private int maxScalingStep = 10;


public int getMaxScalingStep()
{
return maxScalingStep;
}

public PendingTaskBasedWorkerResourceManagementConfig setMaxScalingStep(int maxScalingStep)
{
this.maxScalingStep = maxScalingStep;
return this;
}

public PendingTaskBasedWorkerResourceManagementConfig setWorkerIdleTimeout(Period workerIdleTimeout)
{
super.setWorkerIdleTimeout(workerIdleTimeout);
return this;
}

public PendingTaskBasedWorkerResourceManagementConfig setMaxScalingDuration(Period maxScalingDuration)
{
super.setMaxScalingDuration(maxScalingDuration);
return this;
}

public PendingTaskBasedWorkerResourceManagementConfig setNumEventsToTrack(int numEventsToTrack)
{
super.setNumEventsToTrack(numEventsToTrack);
return this;
}

public PendingTaskBasedWorkerResourceManagementConfig setWorkerVersion(String workerVersion)
{
super.setWorkerVersion(workerVersion);
return this;
}

public PendingTaskBasedWorkerResourceManagementConfig setPendingTaskTimeout(Period pendingTaskTimeout)
{
super.setPendingTaskTimeout(pendingTaskTimeout);
return this;
}

}
Loading

0 comments on commit c29cb7d

Please sign in to comment.