update clhm to 1.3
[cassandra.git] / src / java / org / apache / cassandra / service / CacheService.java
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
9 *
10 * http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18 package org.apache.cassandra.service;
19
20 import java.lang.management.ManagementFactory;
21 import java.util.ArrayList;
22 import java.util.List;
23 import java.util.concurrent.ExecutionException;
24 import java.util.concurrent.Future;
25
26 import javax.management.MBeanServer;
27 import javax.management.ObjectName;
28
29 import org.apache.cassandra.cache.*;
30 import org.apache.cassandra.config.DatabaseDescriptor;
31 import org.apache.cassandra.db.ColumnFamily;
32 import org.apache.cassandra.utils.FBUtilities;
33
34 import org.slf4j.Logger;
35 import org.slf4j.LoggerFactory;
36
37 public class CacheService implements CacheServiceMBean
38 {
39 private static final Logger logger = LoggerFactory.getLogger(CacheService.class);
40
41 public static final String MBEAN_NAME = "org.apache.cassandra.db:type=Caches";
42 public static final int AVERAGE_KEY_CACHE_ROW_SIZE = 48;
43
44 public static enum CacheType
45 {
46 KEY_CACHE("KeyCache"),
47 ROW_CACHE("RowCache");
48
49 private final String name;
50
51 private CacheType(String typeName)
52 {
53 name = typeName;
54 }
55
56 public String toString()
57 {
58 return name;
59 }
60 }
61
62 public final static CacheService instance = new CacheService();
63
64 public final AutoSavingCache<KeyCacheKey, Long> keyCache;
65 public final AutoSavingCache<RowCacheKey, IRowCacheEntry> rowCache;
66
67 private int rowCacheSavePeriod;
68 private int keyCacheSavePeriod;
69
70 private CacheService()
71 {
72 MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
73
74 try
75 {
76 mbs.registerMBean(this, new ObjectName(MBEAN_NAME));
77 }
78 catch (Exception e)
79 {
80 throw new RuntimeException(e);
81 }
82
83 rowCacheSavePeriod = DatabaseDescriptor.getRowCacheSavePeriod();
84 keyCacheSavePeriod = DatabaseDescriptor.getKeyCacheSavePeriod();
85
86 keyCache = initKeyCache();
87 rowCache = initRowCache();
88 }
89
90 /**
91 * We can use Weighers.singleton() because Long can't be leaking memory
92 * @return auto saving cache object
93 */
94 private AutoSavingCache<KeyCacheKey, Long> initKeyCache()
95 {
96 logger.info("Initializing key cache with capacity of {} MBs.", DatabaseDescriptor.getKeyCacheSizeInMB());
97
98 long keyCacheInMemoryCapacity = DatabaseDescriptor.getKeyCacheSizeInMB() * 1024 * 1024;
99
100 // as values are constant size we can use singleton weigher
101 // where 48 = 40 bytes (average size of the key) + 8 bytes (size of value)
102 ICache<KeyCacheKey, Long> kc = ConcurrentLinkedHashCache.create(keyCacheInMemoryCapacity / AVERAGE_KEY_CACHE_ROW_SIZE);
103 AutoSavingCache<KeyCacheKey, Long> keyCache = new AutoSavingCache<KeyCacheKey, Long>(kc, CacheType.KEY_CACHE);
104
105 int keyCacheKeysToSave = DatabaseDescriptor.getKeyCacheKeysToSave();
106
107 logger.info("Scheduling key cache save to each {} seconds (going to save {} keys).",
108 keyCacheSavePeriod,
109 keyCacheKeysToSave == Integer.MAX_VALUE ? "all" : keyCacheKeysToSave);
110
111 keyCache.scheduleSaving(keyCacheSavePeriod, keyCacheKeysToSave);
112
113 return keyCache;
114 }
115
116 /**
117 * @return initialized row cache
118 */
119 private AutoSavingCache<RowCacheKey, IRowCacheEntry> initRowCache()
120 {
121 logger.info("Initializing row cache with capacity of {} MBs and provider {}",
122 DatabaseDescriptor.getRowCacheSizeInMB(),
123 DatabaseDescriptor.getRowCacheProvider().getClass().getName());
124
125 long rowCacheInMemoryCapacity = DatabaseDescriptor.getRowCacheSizeInMB() * 1024 * 1024;
126
127 // cache object
128 ICache<RowCacheKey, IRowCacheEntry> rc = DatabaseDescriptor.getRowCacheProvider().create(rowCacheInMemoryCapacity, true);
129 AutoSavingCache<RowCacheKey, IRowCacheEntry> rowCache = new AutoSavingCache<RowCacheKey, IRowCacheEntry>(rc, CacheType.ROW_CACHE);
130
131 int rowCacheKeysToSave = DatabaseDescriptor.getRowCacheKeysToSave();
132
133 logger.info("Scheduling row cache save to each {} seconds (going to save {} keys).",
134 rowCacheSavePeriod,
135 rowCacheKeysToSave == Integer.MAX_VALUE ? "all" : rowCacheKeysToSave);
136
137 rowCache.scheduleSaving(rowCacheSavePeriod, rowCacheKeysToSave);
138
139 return rowCache;
140 }
141
142 public long getKeyCacheHits()
143 {
144 return keyCache.getHits();
145 }
146
147 public long getRowCacheHits()
148 {
149 return rowCache.getHits();
150 }
151
152 public long getKeyCacheRequests()
153 {
154 return keyCache.getRequests();
155 }
156
157 public long getRowCacheRequests()
158 {
159 return rowCache.getRequests();
160 }
161
162 public double getKeyCacheRecentHitRate()
163 {
164 return keyCache.getRecentHitRate();
165 }
166
167 public double getRowCacheRecentHitRate()
168 {
169 return rowCache.getRecentHitRate();
170 }
171
172 public int getRowCacheSavePeriodInSeconds()
173 {
174 return rowCacheSavePeriod;
175 }
176
177 public void setRowCacheSavePeriodInSeconds(int rcspis)
178 {
179 if (rcspis < 0)
180 throw new RuntimeException("RowCacheSavePeriodInSeconds must be non-negative.");
181
182 rowCacheSavePeriod = rcspis;
183 rowCache.scheduleSaving(rowCacheSavePeriod, DatabaseDescriptor.getRowCacheKeysToSave());
184 }
185
186 public int getKeyCacheSavePeriodInSeconds()
187 {
188 return keyCacheSavePeriod;
189 }
190
191 public void setKeyCacheSavePeriodInSeconds(int kcspis)
192 {
193 if (kcspis < 0)
194 throw new RuntimeException("KeyCacheSavePeriodInSeconds must be non-negative.");
195
196 keyCacheSavePeriod = kcspis;
197 keyCache.scheduleSaving(keyCacheSavePeriod, DatabaseDescriptor.getKeyCacheKeysToSave());
198 }
199
200 public void invalidateKeyCache()
201 {
202 keyCache.clear();
203 }
204
205 public void invalidateRowCache()
206 {
207 rowCache.clear();
208 }
209
210 public long getRowCacheCapacityInBytes()
211 {
212 return rowCache.getCapacity();
213 }
214
215 public long getRowCacheCapacityInMB()
216 {
217 return getRowCacheCapacityInBytes() / 1024 / 1024;
218 }
219
220 public void setRowCacheCapacityInMB(long capacity)
221 {
222 if (capacity < 0)
223 throw new RuntimeException("capacity should not be negative.");
224
225 rowCache.setCapacity(capacity * 1024 * 1024);
226 }
227
228 public long getKeyCacheCapacityInBytes()
229 {
230 return keyCache.getCapacity() * AVERAGE_KEY_CACHE_ROW_SIZE;
231 }
232
233 public long getKeyCacheCapacityInMB()
234 {
235 return getKeyCacheCapacityInBytes() / 1024 / 1024;
236 }
237
238 public void setKeyCacheCapacityInMB(long capacity)
239 {
240 if (capacity < 0)
241 throw new RuntimeException("capacity should not be negative.");
242
243 keyCache.setCapacity(capacity * 1024 * 1024 / 48);
244 }
245
246 public long getRowCacheSize()
247 {
248 return rowCache.weightedSize();
249 }
250
251 public long getKeyCacheSize()
252 {
253 return keyCache.weightedSize() * AVERAGE_KEY_CACHE_ROW_SIZE;
254 }
255
256 public void reduceCacheSizes()
257 {
258 reduceRowCacheSize();
259 reduceKeyCacheSize();
260 }
261
262 public void reduceRowCacheSize()
263 {
264 rowCache.reduceCacheSize();
265 }
266
267 public void reduceKeyCacheSize()
268 {
269 keyCache.reduceCacheSize();
270 }
271
272 public void saveCaches() throws ExecutionException, InterruptedException
273 {
274 List<Future<?>> futures = new ArrayList<Future<?>>(2);
275 logger.debug("submitting cache saves");
276
277 futures.add(keyCache.submitWrite(DatabaseDescriptor.getKeyCacheKeysToSave()));
278 futures.add(rowCache.submitWrite(DatabaseDescriptor.getRowCacheKeysToSave()));
279
280 FBUtilities.waitOnFutures(futures);
281 logger.debug("cache saves completed");
282 }
283 }