review comments

This commit is contained in:
nishantmonu51 2014-03-25 13:27:30 +05:30
parent 6a34c09e05
commit 1c5d2b6dc5
9 changed files with 34 additions and 27 deletions

View File

@ -23,6 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import io.druid.client.cache.Cache;
import io.druid.client.cache.CacheConfig;
import io.druid.client.cache.MapCache;
import io.druid.query.CacheStrategy;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
@ -40,12 +42,15 @@ public class CachePopulatingQueryRunner<T> implements QueryRunner<T>
private final QueryToolChest toolChest;
private final Cache cache;
private final ObjectMapper mapper;
private final CacheConfig cacheConfig;
public CachePopulatingQueryRunner(
String segmentIdentifier,
SegmentDescriptor segmentDescriptor, ObjectMapper mapper,
Cache cache, QueryToolChest toolchest,
QueryRunner<T> base
Cache cache,
QueryToolChest toolchest,
QueryRunner<T> base,
CacheConfig cacheConfig
)
{
this.base = base;
@ -54,6 +59,7 @@ public class CachePopulatingQueryRunner<T> implements QueryRunner<T>
this.toolChest = toolchest;
this.cache = cache;
this.mapper = mapper;
this.cacheConfig = cacheConfig;
}
@Override
@ -63,7 +69,10 @@ public class CachePopulatingQueryRunner<T> implements QueryRunner<T>
final CacheStrategy strategy = toolChest.getCacheStrategy(query);
final boolean populateCache = Boolean.parseBoolean(query.getContextValue("populateCache", "true"))
&& strategy != null && cache.getCacheConfig().isPopulateCache();
&& strategy != null
&& cacheConfig.isPopulateCache()
// historical only populates distributed cache since the cache lookups are done at broker.
&& !(cache instanceof MapCache) ;
Sequence<T> results = base.run(query);
if (populateCache) {
Cache.NamedKey key = CacheUtil.computeSegmentCacheKey(

View File

@ -40,6 +40,7 @@ import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.emitter.EmittingLogger;
import io.druid.client.cache.Cache;
import io.druid.client.cache.CacheConfig;
import io.druid.client.selector.QueryableDruidServer;
import io.druid.client.selector.ServerSelector;
import io.druid.guice.annotations.Smile;
@ -79,19 +80,22 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
private final TimelineServerView serverView;
private final Cache cache;
private final ObjectMapper objectMapper;
private final CacheConfig cacheConfig;
@Inject
public CachingClusteredClient(
QueryToolChestWarehouse warehouse,
TimelineServerView serverView,
Cache cache,
@Smile ObjectMapper objectMapper
@Smile ObjectMapper objectMapper,
CacheConfig cacheConfig
)
{
this.warehouse = warehouse;
this.serverView = serverView;
this.cache = cache;
this.objectMapper = objectMapper;
this.cacheConfig = cacheConfig;
serverView.registerSegmentCallback(
Executors.newFixedThreadPool(
@ -122,9 +126,9 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
final boolean useCache = Boolean.parseBoolean(query.getContextValue("useCache", "true"))
&& strategy != null
&& cache.getCacheConfig().isUseCache();
&& cacheConfig.isUseCache();
final boolean populateCache = Boolean.parseBoolean(query.getContextValue("populateCache", "true"))
&& strategy != null && cache.getCacheConfig().isPopulateCache();
&& strategy != null && cacheConfig.isPopulateCache();
final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false"));

View File

@ -39,8 +39,6 @@ public interface Cache
public CacheStats getStats();
public CacheConfig getCacheConfig();
public class NamedKey
{
final public String namespace;

View File

@ -153,10 +153,4 @@ public class MapCache implements Cache
retVal.rewind();
return retVal;
}
@Override
public CacheConfig getCacheConfig()
{
return config;
}
}

View File

@ -101,7 +101,6 @@ public class MemcachedCache implements Cache
private final AtomicLong timeoutCount = new AtomicLong(0);
private final AtomicLong errorCount = new AtomicLong(0);
private final CacheConfig config;
MemcachedCache(MemcachedClientIF client, MemcachedCacheConfig config) {
Preconditions.checkArgument(config.getMemcachedPrefix().length() <= MAX_PREFIX_LENGTH,
@ -112,7 +111,6 @@ public class MemcachedCache implements Cache
this.expiration = config.getExpiration();
this.client = client;
this.memcachedPrefix = config.getMemcachedPrefix();
this.config = config;
}
@Override
@ -129,12 +127,6 @@ public class MemcachedCache implements Cache
);
}
@Override
public CacheConfig getCacheConfig()
{
return config;
}
@Override
public byte[] get(NamedKey key)
{

View File

@ -31,6 +31,7 @@ import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.client.CachePopulatingQueryRunner;
import io.druid.client.cache.Cache;
import io.druid.client.cache.CacheConfig;
import io.druid.collections.CountingMap;
import io.druid.guice.annotations.Processing;
import io.druid.guice.annotations.Smile;
@ -83,6 +84,7 @@ public class ServerManager implements QuerySegmentWalker
private final CountingMap<String> dataSourceCounts = new CountingMap<String>();
private final Cache cache;
private final ObjectMapper objectMapper;
private final CacheConfig cacheConfig;
@Inject
public ServerManager(
@ -91,7 +93,8 @@ public class ServerManager implements QuerySegmentWalker
ServiceEmitter emitter,
@Processing ExecutorService exec,
@Smile ObjectMapper objectMapper,
Cache cache
Cache cache,
CacheConfig cacheConfig
)
{
this.segmentLoader = segmentLoader;
@ -103,6 +106,7 @@ public class ServerManager implements QuerySegmentWalker
this.objectMapper = objectMapper;
this.dataSources = new HashMap<>();
this.cacheConfig = cacheConfig;
}
public Map<String, Long> getDataSourceSizes()
@ -412,7 +416,8 @@ public class ServerManager implements QuerySegmentWalker
objectMapper,
cache,
toolChest,
new ReferenceCountingSegmentQueryRunner<T>(factory, adapter)
new ReferenceCountingSegmentQueryRunner<T>(factory, adapter),
cacheConfig
)
)
).withWaitMeasuredFromNow(),

View File

@ -1194,7 +1194,8 @@ public class CachingClusteredClientTest
}
},
cache,
jsonMapper
jsonMapper,
new CacheConfig()
);
}

View File

@ -37,6 +37,7 @@ import com.metamx.common.guava.YieldingAccumulator;
import com.metamx.common.guava.YieldingSequenceBase;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.client.cache.CacheConfig;
import io.druid.client.cache.LocalCacheProvider;
import io.druid.granularity.QueryGranularity;
import io.druid.jackson.DefaultObjectMapper;
@ -138,7 +139,8 @@ public class ServerManagerTest
}
},
new NoopServiceEmitter(),
serverManagerExec, new DefaultObjectMapper(), new LocalCacheProvider().get()
serverManagerExec, new DefaultObjectMapper(), new LocalCacheProvider().get(),
new CacheConfig()
);
loadQueryable("test", "1", new Interval("P1d/2011-04-01"));

View File

@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.MoreExecutors;
import com.metamx.common.logger.Logger;
import io.druid.client.cache.CacheConfig;
import io.druid.client.cache.LocalCacheProvider;
import io.druid.concurrent.Execs;
import io.druid.curator.CuratorTestBase;
@ -83,7 +84,8 @@ public class ZkCoordinatorTest extends CuratorTestBase
new NoopServiceEmitter(),
MoreExecutors.sameThreadExecutor(),
new DefaultObjectMapper(),
new LocalCacheProvider().get()
new LocalCacheProvider().get(),
new CacheConfig()
);
final DruidServerMetadata me = new DruidServerMetadata("dummyServer", "dummyHost", 0, "dummyType", "normal", 0);