mirror of https://github.com/apache/druid.git
Fix CachingCostBalancerStrategyFactory's constructor (#4974)
* Fix CachingCostBalancerStrategyFactory's constructor * Fix CachingCostBalancerStrategyFactory not registered in Lifecycle
This commit is contained in:
parent
5fc6891404
commit
26b87c9f8e
|
@ -19,16 +19,17 @@
|
||||||
|
|
||||||
package io.druid.server.coordinator;
|
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.base.Preconditions;
|
||||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
import com.google.inject.Inject;
|
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import io.druid.client.ServerInventoryView;
|
import io.druid.client.ServerInventoryView;
|
||||||
import io.druid.client.ServerView;
|
import io.druid.client.ServerView;
|
||||||
import io.druid.java.util.common.concurrent.Execs;
|
import io.druid.java.util.common.concurrent.Execs;
|
||||||
import io.druid.concurrent.LifecycleLock;
|
import io.druid.concurrent.LifecycleLock;
|
||||||
import io.druid.guice.ManageLifecycle;
|
|
||||||
import io.druid.java.util.common.ISE;
|
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.LifecycleStart;
|
||||||
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
||||||
import io.druid.server.coordination.DruidServerMetadata;
|
import io.druid.server.coordination.DruidServerMetadata;
|
||||||
|
@ -43,7 +44,6 @@ import java.util.concurrent.RejectedExecutionException;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
@ManageLifecycle
|
|
||||||
public class CachingCostBalancerStrategyFactory implements BalancerStrategyFactory
|
public class CachingCostBalancerStrategyFactory implements BalancerStrategyFactory
|
||||||
{
|
{
|
||||||
private static final EmittingLogger LOG = new EmittingLogger(CachingCostBalancerStrategyFactory.class);
|
private static final EmittingLogger LOG = new EmittingLogger(CachingCostBalancerStrategyFactory.class);
|
||||||
|
@ -55,10 +55,16 @@ public class CachingCostBalancerStrategyFactory implements BalancerStrategyFacto
|
||||||
private final ClusterCostCache.Builder clusterCostCacheBuilder = ClusterCostCache.builder();
|
private final ClusterCostCache.Builder clusterCostCacheBuilder = ClusterCostCache.builder();
|
||||||
private volatile boolean initialized = false;
|
private volatile boolean initialized = false;
|
||||||
|
|
||||||
@Inject
|
@JsonCreator
|
||||||
public CachingCostBalancerStrategyFactory(ServerInventoryView serverInventoryView)
|
public CachingCostBalancerStrategyFactory(
|
||||||
|
@JacksonInject ServerInventoryView serverInventoryView,
|
||||||
|
@JacksonInject Lifecycle lifecycle
|
||||||
|
) throws Exception
|
||||||
{
|
{
|
||||||
this.serverInventoryView = Preconditions.checkNotNull(serverInventoryView);
|
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
|
@LifecycleStart
|
||||||
|
@ -123,7 +129,7 @@ public class CachingCostBalancerStrategyFactory implements BalancerStrategyFacto
|
||||||
@Override
|
@Override
|
||||||
public BalancerStrategy createBalancerStrategy(final ListeningExecutorService exec)
|
public BalancerStrategy createBalancerStrategy(final ListeningExecutorService exec)
|
||||||
{
|
{
|
||||||
if (!lifecycleLock.awaitStarted()) {
|
if (!lifecycleLock.awaitStarted(1, TimeUnit.MINUTES)) {
|
||||||
throw new ISE("CachingCostBalancerStrategyFactory is not started");
|
throw new ISE("CachingCostBalancerStrategyFactory is not started");
|
||||||
}
|
}
|
||||||
if (initialized) {
|
if (initialized) {
|
||||||
|
|
Loading…
Reference in New Issue