Add futures to SQLMetadataSegmentManager and SQLMetadataRuleManager

This commit is contained in:
Charles Allen 2015-05-14 16:46:20 -07:00
parent 3c3db7229c
commit 051c3ccede
2 changed files with 67 additions and 47 deletions

View File

@ -25,9 +25,11 @@ import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
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.Pair;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
@ -41,7 +43,6 @@ import io.druid.guice.annotations.Json;
import io.druid.server.coordinator.rules.ForeverLoadRule;
import io.druid.server.coordinator.rules.Rule;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.skife.jdbi.v2.FoldController;
import org.skife.jdbi.v2.Folder3;
import org.skife.jdbi.v2.Handle;
@ -58,7 +59,7 @@ import java.sql.SQLException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
/**
@ -134,12 +135,13 @@ public class SQLMetadataRuleManager implements MetadataRuleManager
private final AtomicReference<ImmutableMap<String, List<Rule>>> rules;
private final AuditManager auditManager;
private volatile ScheduledExecutorService exec;
private final Object lock = new Object();
private volatile boolean started = false;
private volatile ListeningScheduledExecutorService exec = null;
private volatile ListenableFuture<?> future = null;
@Inject
public SQLMetadataRuleManager(
@Json ObjectMapper jsonMapper,
@ -168,21 +170,26 @@ public class SQLMetadataRuleManager implements MetadataRuleManager
return;
}
this.exec = Execs.scheduledSingleThreaded("DatabaseRuleManager-Exec--%d");
exec = MoreExecutors.listeningDecorator(Execs.scheduledSingleThreaded("DatabaseRuleManager-Exec--%d"));
createDefaultRule(dbi, getRulesTable(), config.get().getDefaultRule(), jsonMapper);
ScheduledExecutors.scheduleWithFixedDelay(
exec,
new Duration(0),
config.get().getPollDuration().toStandardDuration(),
future = exec.scheduleWithFixedDelay(
new Runnable()
{
@Override
public void run()
{
try {
poll();
}
catch (Exception e) {
log.error(e, "uncaught exception in rule manager polling thread");
}
}
},
0,
config.get().getPollDuration().toStandardDuration().getMillis(),
TimeUnit.MILLISECONDS
);
started = true;
@ -199,6 +206,8 @@ public class SQLMetadataRuleManager implements MetadataRuleManager
rules.set(ImmutableMap.<String, List<Rule>>of());
future.cancel(false);
future = null;
started = false;
exec.shutdownNow();
exec = null;
@ -235,7 +244,9 @@ public class SQLMetadataRuleManager implements MetadataRuleManager
return Pair.of(
r.getString("dataSource"),
jsonMapper.<List<Rule>>readValue(
r.getBytes("payload"), new TypeReference<List<Rule>>(){}
r.getBytes("payload"), new TypeReference<List<Rule>>()
{
}
)
);
}

View File

@ -22,9 +22,11 @@ import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
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.MapUtils;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
@ -56,7 +58,7 @@ import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
/**
@ -74,7 +76,8 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
private final AtomicReference<ConcurrentHashMap<String, DruidDataSource>> dataSources;
private final IDBI dbi;
private volatile ScheduledExecutorService exec;
private volatile ListeningScheduledExecutorService exec = null;
private volatile ListenableFuture<?> future = null;
private volatile boolean started = false;
@ -103,23 +106,27 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
return;
}
this.exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d");
exec = MoreExecutors.listeningDecorator(Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d"));
final Duration delay = config.get().getPollDuration().toStandardDuration();
ScheduledExecutors.scheduleWithFixedDelay(
exec,
new Duration(0),
delay,
future = exec.scheduleWithFixedDelay(
new Runnable()
{
@Override
public void run()
{
try {
poll();
}
catch (Exception e) {
log.error(e, "uncaught exception in segment manager polling thread");
}
}
},
0,
delay.getMillis(),
TimeUnit.MILLISECONDS
);
started = true;
}
}
@ -134,6 +141,8 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
started = false;
dataSources.set(new ConcurrentHashMap<String, DruidDataSource>());
future.cancel(false);
future = null;
exec.shutdownNow();
exec = null;
}