Skip to content
Merged
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 @@ -19,16 +19,17 @@

package io.druid.server.coordinator;

import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.inject.Inject;
import com.metamx.emitter.EmittingLogger;
import io.druid.client.ServerInventoryView;
import io.druid.client.ServerView;
import io.druid.java.util.common.concurrent.Execs;
import io.druid.concurrent.LifecycleLock;
import io.druid.guice.ManageLifecycle;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.lifecycle.Lifecycle;
import io.druid.java.util.common.lifecycle.LifecycleStart;
import io.druid.java.util.common.lifecycle.LifecycleStop;
import io.druid.server.coordination.DruidServerMetadata;
Expand All @@ -43,7 +44,6 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;

@ManageLifecycle
public class CachingCostBalancerStrategyFactory implements BalancerStrategyFactory
{
private static final EmittingLogger LOG = new EmittingLogger(CachingCostBalancerStrategyFactory.class);
Expand All @@ -55,10 +55,16 @@ public class CachingCostBalancerStrategyFactory implements BalancerStrategyFacto
private final ClusterCostCache.Builder clusterCostCacheBuilder = ClusterCostCache.builder();
private volatile boolean initialized = false;

@Inject
public CachingCostBalancerStrategyFactory(ServerInventoryView serverInventoryView)
@JsonCreator
public CachingCostBalancerStrategyFactory(
@JacksonInject ServerInventoryView serverInventoryView,
@JacksonInject Lifecycle lifecycle
) throws Exception
{
this.serverInventoryView = Preconditions.checkNotNull(serverInventoryView);
// Adding to lifecycle dynamically because couldn't use @ManageLifecycle on the class,
// see https://github.com/druid-io/druid/issues/4980
lifecycle.addMaybeStartManagedInstance(this);
}

@LifecycleStart
Expand Down Expand Up @@ -123,7 +129,7 @@ public void stop()
@Override
public BalancerStrategy createBalancerStrategy(final ListeningExecutorService exec)
{
if (!lifecycleLock.awaitStarted()) {
if (!lifecycleLock.awaitStarted(1, TimeUnit.MINUTES)) {
throw new ISE("CachingCostBalancerStrategyFactory is not started");
}
if (initialized) {
Expand Down