Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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()));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

any reason we truncate the time here?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I kept the existing logic and just abstracted out from SimpleResourceManagementStrategy. I believe its related to the introduction of origin time in resource management scheduler.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@fjy any clues as to why you originally added originTime and timestamp truncation as opposed to just taking now + duration as the next timestamp?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@fjy: any thoughts ?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@xvrl: I checked the code in more detail, it seems to be making sure that for every overlord restart the scheduler runs at same time. I think its helpful in case someone wants to terminate workers only at specific fixed absolute intervals.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It was made for the way Amazon prices, where I believe they only charge at the top of the hour


ScheduledExecutors.scheduleAtFixedRate(
exec,
new Duration(System.currentTimeMillis(), startTime),
resourceManagementSchedulerConfig.getTerminatePeriod().toStandardDuration(),
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

minor nit, but toStandardDuration could potentially throw an UnsupportedOperationException, we might want to convert to duration upfront in the config.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I kept the existing logic unchanged, just abstracted out the code.

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