Skip to content

Commit d56e7f2

Browse files
authored
HBASE-27796 Improve MemcachedBlockCache (#5181)
Track and log better stats. Fix use of memcached API such that we are not accidentally immediately expiring keys. Use a default retention period of 30 days, the max per memcached spec. Use set instead of add to store keys. Gracefully handle failures to cache and read timeouts. Add unit tests using jmemcached as a test dependency. Signed-off-by: Viraj Jasani <vjasani@apache.org>
1 parent 31c4aea commit d56e7f2

3 files changed

Lines changed: 287 additions & 24 deletions

File tree

hbase-external-blockcache/pom.xml

Lines changed: 57 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,63 @@
5151
</dependency>
5252
<dependency>
5353
<groupId>org.slf4j</groupId>
54-
<artifactId>slf4j-api</artifactId>
54+
<artifactId>jcl-over-slf4j</artifactId>
55+
<scope>test</scope>
56+
</dependency>
57+
<dependency>
58+
<groupId>org.slf4j</groupId>
59+
<artifactId>jul-to-slf4j</artifactId>
60+
<scope>test</scope>
61+
</dependency>
62+
<dependency>
63+
<groupId>org.apache.logging.log4j</groupId>
64+
<artifactId>log4j-api</artifactId>
65+
<scope>test</scope>
66+
</dependency>
67+
<dependency>
68+
<groupId>org.apache.logging.log4j</groupId>
69+
<artifactId>log4j-core</artifactId>
70+
<scope>test</scope>
71+
</dependency>
72+
<dependency>
73+
<groupId>org.apache.logging.log4j</groupId>
74+
<artifactId>log4j-slf4j-impl</artifactId>
75+
<scope>test</scope>
76+
</dependency>
77+
<dependency>
78+
<groupId>org.apache.logging.log4j</groupId>
79+
<artifactId>log4j-1.2-api</artifactId>
80+
<scope>test</scope>
81+
</dependency>
82+
<dependency>
83+
<groupId>org.apache.hbase</groupId>
84+
<artifactId>hbase-logging</artifactId>
85+
<type>test-jar</type>
86+
<scope>test</scope>
87+
</dependency>
88+
<dependency>
89+
<groupId>org.apache.hbase</groupId>
90+
<artifactId>hbase-common</artifactId>
91+
<type>test-jar</type>
92+
<scope>test</scope>
93+
</dependency>
94+
<dependency>
95+
<groupId>org.apache.hbase</groupId>
96+
<artifactId>hbase-zookeeper</artifactId>
97+
<type>test-jar</type>
98+
<scope>test</scope>
99+
</dependency>
100+
<dependency>
101+
<groupId>org.apache.hbase</groupId>
102+
<artifactId>hbase-server</artifactId>
103+
<type>test-jar</type>
104+
<scope>test</scope>
105+
</dependency>
106+
<dependency>
107+
<groupId>com.thimbleware.jmemcached</groupId>
108+
<artifactId>jmemcached-core</artifactId>
109+
<version>1.0.0</version>
110+
<scope>test</scope>
55111
</dependency>
56112
<dependency>
57113
<groupId>junit</groupId>

hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java

Lines changed: 106 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -27,35 +27,47 @@
2727
import java.util.List;
2828
import java.util.NoSuchElementException;
2929
import java.util.concurrent.ExecutionException;
30+
import java.util.concurrent.Executors;
31+
import java.util.concurrent.ScheduledExecutorService;
32+
import java.util.concurrent.TimeUnit;
33+
import java.util.concurrent.atomic.AtomicLong;
3034
import net.spy.memcached.CachedData;
3135
import net.spy.memcached.ConnectionFactoryBuilder;
3236
import net.spy.memcached.FailureMode;
3337
import net.spy.memcached.MemcachedClient;
38+
import net.spy.memcached.OperationTimeoutException;
3439
import net.spy.memcached.transcoders.Transcoder;
3540
import org.apache.hadoop.conf.Configuration;
36-
import org.apache.hadoop.hbase.HConstants;
3741
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
3842
import org.apache.hadoop.hbase.nio.ByteBuff;
3943
import org.apache.hadoop.hbase.nio.SingleByteBuff;
4044
import org.apache.hadoop.hbase.trace.TraceUtil;
4145
import org.apache.hadoop.hbase.util.Addressing;
46+
import org.apache.hadoop.util.StringUtils;
4247
import org.apache.yetus.audience.InterfaceAudience;
4348
import org.slf4j.Logger;
4449
import org.slf4j.LoggerFactory;
4550

51+
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
52+
4653
/**
4754
* Class to store blocks into memcached. This should only be used on a cluster of Memcached daemons
4855
* that are tuned well and have a good network connection to the HBase regionservers. Any other use
4956
* will likely slow down HBase greatly.
5057
*/
5158
@InterfaceAudience.Private
59+
@SuppressWarnings("FutureReturnValueIgnored")
5260
public class MemcachedBlockCache implements BlockCache {
5361
private static final Logger LOG = LoggerFactory.getLogger(MemcachedBlockCache.class.getName());
5462

5563
// Some memcache versions won't take more than 1024 * 1024. So set the limit below
5664
// that just in case this client is used with those versions.
5765
public static final int MAX_SIZE = 1020 * 1024;
5866

67+
// Start memcached with -I <MAX_SIZE> to ensure it has the ability to store blocks of this size
68+
public static final int MAX_TIME = 60 * 60 * 24 * 30; // 30 days, max allowed per the memcached
69+
// spec
70+
5971
// Config key for what memcached servers to use.
6072
// They should be specified in a comma sperated list with ports.
6173
// like:
@@ -67,10 +79,20 @@ public class MemcachedBlockCache implements BlockCache {
6779
public static final String MEMCACHED_OPTIMIZE_KEY = "hbase.cache.memcached.spy.optimze";
6880
public static final long MEMCACHED_DEFAULT_TIMEOUT = 500;
6981
public static final boolean MEMCACHED_OPTIMIZE_DEFAULT = false;
82+
public static final int STAT_THREAD_PERIOD = 60 * 5;
7083

7184
private final MemcachedClient client;
7285
private final HFileBlockTranscoder tc = new HFileBlockTranscoder();
7386
private final CacheStats cacheStats = new CacheStats("MemcachedBlockCache");
87+
private final AtomicLong cachedCount = new AtomicLong();
88+
private final AtomicLong notCachedCount = new AtomicLong();
89+
private final AtomicLong cacheErrorCount = new AtomicLong();
90+
private final AtomicLong timeoutCount = new AtomicLong();
91+
92+
/** Statistics thread schedule pool (for heavy debugging, could remove) */
93+
private transient final ScheduledExecutorService scheduleThreadPool =
94+
Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
95+
.setNameFormat("MemcachedBlockCacheStatsExecutor").setDaemon(true).build());
7496

7597
public MemcachedBlockCache(Configuration c) throws IOException {
7698
LOG.info("Creating MemcachedBlockCache");
@@ -80,18 +102,15 @@ public MemcachedBlockCache(Configuration c) throws IOException {
80102
boolean optimize = c.getBoolean(MEMCACHED_OPTIMIZE_KEY, MEMCACHED_OPTIMIZE_DEFAULT);
81103

82104
ConnectionFactoryBuilder builder =
83-
// Cap the max time before anything times out
84105
new ConnectionFactoryBuilder().setOpTimeout(opTimeout).setOpQueueMaxBlockTime(queueTimeout)
85-
// Don't keep threads around past the end of days.
86106
.setFailureMode(FailureMode.Redistribute).setShouldOptimize(optimize).setDaemon(true)
87-
.setUseNagleAlgorithm(false) // Ain't nobody got time for that
88-
.setReadBufferSize(HConstants.DEFAULT_BLOCKSIZE * 4 * 1024); // Much larger just in case
107+
.setUseNagleAlgorithm(false).setReadBufferSize(MAX_SIZE);
89108

90-
// Assume only the localhost is serving memecached.
109+
// Assume only the localhost is serving memcached.
91110
// A la mcrouter or co-locating memcached with split regionservers.
92111
//
93-
// If this config is a pool of memecached servers they will all be used according to the
94-
// default hashing scheme defined by the memcache client. Spy Memecache client in this
112+
// If this config is a pool of memcached servers they will all be used according to the
113+
// default hashing scheme defined by the memcached client. Spy Memecache client in this
95114
// case.
96115
String serverListString = c.get(MEMCACHED_CONFIG_KEY, "localhost:11211");
97116
String[] servers = serverListString.split(",");
@@ -104,29 +123,36 @@ public MemcachedBlockCache(Configuration c) throws IOException {
104123
}
105124

106125
client = new MemcachedClient(builder.build(), serverAddresses);
126+
this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this), STAT_THREAD_PERIOD,
127+
STAT_THREAD_PERIOD, TimeUnit.SECONDS);
107128
}
108129

109130
@Override
110131
public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
111132
cacheBlock(cacheKey, buf);
112133
}
113134

114-
@SuppressWarnings("FutureReturnValueIgnored")
115135
@Override
116136
public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
117137
if (buf instanceof HFileBlock) {
118-
client.add(cacheKey.toString(), MAX_SIZE, (HFileBlock) buf, tc).addListener(f -> {
138+
if (buf.getSerializedLength() > MAX_SIZE) {
139+
LOG.debug("Block of type {} with key {} is too large, size={}, max={}, will not cache",
140+
buf.getClass(), cacheKey, buf.getSerializedLength(), MAX_SIZE);
141+
notCachedCount.incrementAndGet();
142+
return;
143+
}
144+
client.set(cacheKey.toString(), MAX_TIME, (HFileBlock) buf, tc).addListener(f -> {
119145
try {
120146
f.get();
121-
} catch (ExecutionException e) {
122-
LOG.warn("Failed to cache block", e);
147+
cachedCount.incrementAndGet();
148+
} catch (Exception e) {
149+
LOG.warn("Failed to cache block with key " + cacheKey, e);
150+
cacheErrorCount.incrementAndGet();
123151
}
124152
});
125153
} else {
126-
if (LOG.isDebugEnabled()) {
127-
LOG.debug(
128-
"MemcachedBlockCache can not cache Cacheable's of type " + buf.getClass().toString());
129-
}
154+
LOG.debug("Can not cache Cacheables of type {} with key {}", buf.getClass(), cacheKey);
155+
notCachedCount.incrementAndGet();
130156
}
131157
}
132158

@@ -139,17 +165,25 @@ public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repea
139165
try (Scope traceScope = span.makeCurrent()) {
140166
result = client.get(cacheKey.toString(), tc);
141167
} catch (Exception e) {
142-
// Catch a pretty broad set of exceptions to limit any changes in the memecache client
168+
// Catch a pretty broad set of exceptions to limit any changes in the memcache client
143169
// and how it handles failures from leaking into the read path.
144-
if (LOG.isDebugEnabled()) {
145-
LOG.debug(
146-
"Exception pulling from memcached [ " + cacheKey.toString() + " ]. Treating as a miss.",
147-
e);
170+
if (
171+
(e instanceof OperationTimeoutException) || ((e instanceof RuntimeException)
172+
&& (e.getCause() instanceof OperationTimeoutException))
173+
) {
174+
timeoutCount.incrementAndGet();
175+
if (LOG.isDebugEnabled()) {
176+
LOG.debug("Timeout getting key " + cacheKey.toString(), e);
177+
}
178+
} else {
179+
cacheErrorCount.incrementAndGet();
180+
if (LOG.isDebugEnabled()) {
181+
LOG.debug("Exception getting key " + cacheKey.toString(), e);
182+
}
148183
}
149184
result = null;
150185
} finally {
151186
span.end();
152-
// Update stats if this request doesn't have it turned off 100% of the time
153187
if (updateCacheMetrics) {
154188
if (result == null) {
155189
cacheStats.miss(caching, cacheKey.isPrimary(), cacheKey.getBlockType());
@@ -158,7 +192,6 @@ public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repea
158192
}
159193
}
160194
}
161-
162195
return result;
163196
}
164197

@@ -194,6 +227,22 @@ public CacheStats getStats() {
194227
@Override
195228
public void shutdown() {
196229
client.shutdown();
230+
this.scheduleThreadPool.shutdown();
231+
for (int i = 0; i < 10; i++) {
232+
if (!this.scheduleThreadPool.isShutdown()) {
233+
try {
234+
Thread.sleep(10);
235+
} catch (InterruptedException e) {
236+
LOG.warn("Interrupted while sleeping");
237+
Thread.currentThread().interrupt();
238+
break;
239+
}
240+
}
241+
}
242+
if (!this.scheduleThreadPool.isShutdown()) {
243+
List<Runnable> runnables = this.scheduleThreadPool.shutdownNow();
244+
LOG.debug("Still running " + runnables);
245+
}
197246
}
198247

199248
@Override
@@ -289,4 +338,38 @@ public int getMaxSize() {
289338
return MAX_SIZE;
290339
}
291340
}
341+
342+
private static class StatisticsThread extends Thread {
343+
344+
private final MemcachedBlockCache c;
345+
346+
public StatisticsThread(MemcachedBlockCache c) {
347+
super("MemcachedBlockCacheStats");
348+
setDaemon(true);
349+
this.c = c;
350+
}
351+
352+
@Override
353+
public void run() {
354+
c.logStats();
355+
}
356+
357+
}
358+
359+
public void logStats() {
360+
LOG.info("cached=" + cachedCount.get() + ", notCached=" + notCachedCount.get()
361+
+ ", cacheErrors=" + cacheErrorCount.get() + ", timeouts=" + timeoutCount.get() + ", reads="
362+
+ cacheStats.getRequestCount() + ", " + "hits=" + cacheStats.getHitCount() + ", hitRatio="
363+
+ (cacheStats.getHitCount() == 0
364+
? "0"
365+
: (StringUtils.formatPercent(cacheStats.getHitRatio(), 2) + ", "))
366+
+ "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " + "cachingHits="
367+
+ cacheStats.getHitCachingCount() + ", " + "cachingHitsRatio="
368+
+ (cacheStats.getHitCachingCount() == 0
369+
? "0,"
370+
: (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2) + ", "))
371+
+ "evictions=" + cacheStats.getEvictionCount() + ", " + "evicted="
372+
+ cacheStats.getEvictedCount() + ", " + "evictedPerRun=" + cacheStats.evictedPerEviction());
373+
}
374+
292375
}

0 commit comments

Comments
 (0)