fix according to comments in code review

This commit is contained in:
fjy 2013-07-23 16:21:45 -07:00
parent 94afe72133
commit e4c2a2c705
4 changed files with 13 additions and 7 deletions

View File

@ -51,6 +51,7 @@ public class BatchServerInventoryView extends ServerInventoryView<Set<DataSegmen
{
super(
config,
log,
new InventoryManagerConfig()
{
@Override

View File

@ -27,6 +27,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.MapMaker;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.curator.inventory.CuratorInventoryManager;
import com.metamx.druid.curator.inventory.CuratorInventoryManagerStrategy;
import com.metamx.druid.curator.inventory.InventoryManagerConfig;
@ -45,19 +46,19 @@ import java.util.concurrent.atomic.AtomicBoolean;
*/
public abstract class ServerInventoryView<InventoryType> implements ServerView, InventoryView
{
protected static final EmittingLogger log = new EmittingLogger(ServerInventoryView.class);
private ServerInventoryViewConfig config;
private final ServerInventoryViewConfig config;
private final Logger log;
private final CuratorInventoryManager<DruidServer, InventoryType> inventoryManager;
private final AtomicBoolean started = new AtomicBoolean(false);
protected final ConcurrentMap<ServerCallback, Executor> serverCallbacks = new MapMaker().makeMap();
protected final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks = new MapMaker().makeMap();
private final ConcurrentMap<ServerCallback, Executor> serverCallbacks = new MapMaker().makeMap();
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks = new MapMaker().makeMap();
protected static final Map<String, Integer> removedSegments = new MapMaker().makeMap();
private final Map<String, Integer> removedSegments = new MapMaker().makeMap();
public ServerInventoryView(
final ServerInventoryViewConfig config,
final Logger log,
final InventoryManagerConfig inventoryManagerConfig,
final CuratorFramework curator,
final ExecutorService exec,
@ -66,6 +67,7 @@ public abstract class ServerInventoryView<InventoryType> implements ServerView,
)
{
this.config = config;
this.log = log;
this.inventoryManager = new CuratorInventoryManager<DruidServer, InventoryType>(
curator,
inventoryManagerConfig,

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.druid.curator.inventory.InventoryManagerConfig;
import com.metamx.druid.initialization.ZkPathsConfig;
import com.metamx.emitter.EmittingLogger;
import org.apache.curator.framework.CuratorFramework;
import java.util.concurrent.ExecutorService;
@ -31,6 +32,8 @@ import java.util.concurrent.ExecutorService;
*/
public class SingleServerInventoryView extends ServerInventoryView<DataSegment>
{
private static final EmittingLogger log = new EmittingLogger(SingleServerInventoryView.class);
public SingleServerInventoryView(
final ServerInventoryViewConfig config,
final ZkPathsConfig zkPaths,
@ -41,6 +44,7 @@ public class SingleServerInventoryView extends ServerInventoryView<DataSegment>
{
super(
config,
log,
new InventoryManagerConfig()
{
@Override

View File

@ -197,7 +197,6 @@ public class Initialization
.connectString(curatorConfig.getZkHosts())
.sessionTimeoutMs(curatorConfig.getZkSessionTimeoutMs())
.retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30))
// Don't compress stuff written just yet, need to get code deployed first.
.compressionProvider(new PotentiallyGzippedCompressionProvider(curatorConfig.enableCompression()))
.build();