Add useL2 and populateL2 configs to HybridCache (#5088)

* Add useL2 and populateL2 configs to HybridCache

* typo
This commit is contained in:
Roman Leventov 2017-11-20 19:57:05 -03:00 committed by Himanshu
parent e115da39df
commit dbb37b727d
8 changed files with 120 additions and 11 deletions

View File

@ -30,6 +30,16 @@
<inspection_tool class="EqualsBetweenInconvertibleTypes" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="EqualsUsesNonFinalVariable" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="EqualsWithItself" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="FieldCanBeLocal" enabled="true" level="WARNING" enabled_by_default="true">
<option name="EXCLUDE_ANNOS">
<value>
<list size="1">
<item index="0" class="java.lang.String" itemvalue="com.fasterxml.jackson.annotation.JsonProperty" />
</list>
</value>
</option>
<option name="IGNORE_FIELDS_USED_IN_MULTIPLE_METHODS" value="true" />
</inspection_tool>
<inspection_tool class="ForCanBeForeach" enabled="true" level="WARNING" enabled_by_default="true">
<scope name="NonGeneratedFiles" level="ERROR" enabled="true">
<option name="REPORT_INDEXED_LOOP" value="true" />

View File

@ -0,0 +1,6 @@
<component name="InspectionProjectProfileManager">
<settings>
<option name="PROJECT_PROFILE" value="Druid" />
<version value="1.0" />
</settings>
</component>

View File

@ -101,3 +101,5 @@ If there is an L1 miss and L2 hit, it will also populate L1.
|`druid.cache.l2.type`|type of cache to use for L2 cache. See `druid.cache.type` configuration for valid types.|`local`|
|`druid.cache.l1.*`|Any property valid for the given type of L1 cache can be set using this prefix. For instance, if you are using a `local` L1 cache, specify `druid.cache.l1.sizeInBytes` to set its size.|defaults are the same as for the given cache type.|
|`druid.cache.l2.*`|Prefix for L2 cache settings, see description for L1.|defaults are the same as for the given cache type.|
|`druid.cache.useL2`|A boolean indicating whether to query L2 cache, if it's a miss in L1. It makes sense to configure this to `false` on historical nodes, if L2 is a remote cache like `memcached`, and this cache also used on brokers, because in this case if a query reached historical it means that a broker didn't find corresponding results in the same remote cache, so a query to the remote cache from historical is guaranteed to be a miss.|`true`|
|`druid.cache.populateL2`|A boolean indicating whether to put results into L2 cache.|`true`|

View File

@ -24,6 +24,7 @@ import com.google.common.primitives.Ints;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.java.util.common.StringUtils;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Map;
@ -32,6 +33,7 @@ import java.util.Map;
*/
public interface Cache
{
@Nullable
byte[] get(NamedKey key);
void put(NamedKey key, byte[] value);

View File

@ -22,50 +22,71 @@ package io.druid.client.cache;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
public class HybridCache implements Cache
{
private static final Logger log = new Logger(HybridCache.class);
private final HybridCacheConfig config;
private final Cache level1;
private final Cache level2;
private final AtomicLong hitCount = new AtomicLong(0);
private final AtomicLong missCount = new AtomicLong(0);
public HybridCache(Cache level1, Cache level2)
public HybridCache(HybridCacheConfig config, Cache level1, Cache level2)
{
this.config = config;
log.info("Config: %s", config);
this.level1 = level1;
this.level2 = level2;
}
@Nullable
@Override
public byte[] get(NamedKey key)
{
byte[] res = level1.get(key);
if (res == null) {
res = level2.get(key);
res = getL2(key);
if (res != null) {
level1.put(key, res);
hitCount.incrementAndGet();
return res;
}
missCount.incrementAndGet();
}
if (res != null) {
hitCount.incrementAndGet();
}
return res;
} else {
missCount.incrementAndGet();
return null;
}
}
@Nullable
private byte[] getL2(NamedKey key)
{
if (config.getUseL2()) {
return level2.get(key);
} else {
return null;
}
}
@Override
public void put(NamedKey key, byte[] value)
{
level1.put(key, value);
if (config.getPopulateL2()) {
level2.put(key, value);
}
}
@Override
public Map<NamedKey, byte[]> getBulk(Iterable<NamedKey> keys)
@ -77,7 +98,7 @@ public class HybridCache implements Cache
remaining = Sets.difference(remaining, res.keySet());
if (!remaining.isEmpty()) {
Map<NamedKey, byte[]> res2 = level2.getBulk(remaining);
Map<NamedKey, byte[]> res2 = getBulkL2(remaining);
for (Map.Entry<NamedKey, byte[]> entry : res2.entrySet()) {
level1.put(entry.getKey(), entry.getValue());
}
@ -94,6 +115,15 @@ public class HybridCache implements Cache
return res;
}
private Map<NamedKey, byte[]> getBulkL2(Iterable<NamedKey> keys)
{
if (config.getUseL2()) {
return level2.getBulk(keys);
} else {
return Collections.emptyMap();
}
}
@Override
public void close(String namespace)
{

View File

@ -0,0 +1,50 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.client.cache;
import com.fasterxml.jackson.annotation.JsonProperty;
public class HybridCacheConfig
{
@JsonProperty
private boolean useL2 = true;
@JsonProperty
private boolean populateL2 = true;
public boolean getUseL2()
{
return useL2;
}
public boolean getPopulateL2()
{
return populateL2;
}
@Override
public String toString()
{
return "HybridCacheConfig{" +
"useL2=" + useL2 +
", populateL2=" + populateL2 +
'}';
}
}

View File

@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.google.common.base.Preconditions;
import com.google.inject.name.Named;
public class HybridCacheProvider implements CacheProvider
public class HybridCacheProvider extends HybridCacheConfig implements CacheProvider
{
final CacheProvider level1;
final CacheProvider level2;
@ -37,11 +37,17 @@ public class HybridCacheProvider implements CacheProvider
{
this.level1 = Preconditions.checkNotNull(level1, "l1 cache not specified for hybrid cache");
this.level2 = Preconditions.checkNotNull(level2, "l2 cache not specified for hybrid cache");
if (!getUseL2() && !getPopulateL2()) {
throw new IllegalStateException(
"Doesn't make sense to use Hybrid cache with both use and populate disabled for L2, "
+ "use just L1 cache in this case"
);
}
}
@Override
public Cache get()
{
return new HybridCache(level1.get(), level2.get());
return new HybridCache(this, level1.get(), level2.get());
}
}

View File

@ -49,6 +49,7 @@ public class HybridCacheTest
System.setProperty(prefix + ".type", "hybrid");
System.setProperty(prefix + ".l1.type", "local");
System.setProperty(prefix + ".l2.type", "memcached");
System.setProperty(prefix + ".useL2", "false");
System.setProperty(prefix + ".l2.hosts", "localhost:11711");
final Injector injector = Initialization.makeInjectorWithModules(
@ -74,6 +75,8 @@ public class HybridCacheTest
Assert.assertNotNull(cache);
Assert.assertFalse(cache.isLocal());
Assert.assertFalse(((HybridCacheProvider) cacheProvider).getUseL2());
Assert.assertTrue(((HybridCacheProvider) cacheProvider).getPopulateL2());
Assert.assertEquals(LocalCacheProvider.class, ((HybridCacheProvider) cacheProvider).level1.getClass());
Assert.assertEquals(MemcachedCacheProvider.class, ((HybridCacheProvider) cacheProvider).level2.getClass());
}
@ -83,7 +86,7 @@ public class HybridCacheTest
{
final MapCache l1 = new MapCache(new ByteCountingLRUMap(1024 * 1024));
final MapCache l2 = new MapCache(new ByteCountingLRUMap(1024 * 1024));
HybridCache cache = new HybridCache(l1, l2);
HybridCache cache = new HybridCache(new HybridCacheConfig(), l1, l2);
final Cache.NamedKey key1 = new Cache.NamedKey("a", HI);
final Cache.NamedKey key2 = new Cache.NamedKey("b", HI);