|
@@ -26,32 +26,31 @@ import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
|
|
|
+import org.ehcache.Cache;
|
|
|
+import org.ehcache.CacheManager;
|
|
|
+import org.ehcache.config.builders.CacheConfigurationBuilder;
|
|
|
+import org.ehcache.config.builders.CacheManagerBuilder;
|
|
|
+import org.ehcache.config.builders.ExpiryPolicyBuilder;
|
|
|
+import org.ehcache.config.builders.ResourcePoolsBuilder;
|
|
|
+import org.ehcache.expiry.ExpiryPolicy;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import javax.cache.Cache;
|
|
|
-import javax.cache.CacheManager;
|
|
|
-import javax.cache.Caching;
|
|
|
-import javax.cache.configuration.FactoryBuilder;
|
|
|
-import javax.cache.configuration.MutableConfiguration;
|
|
|
-import javax.cache.expiry.CreatedExpiryPolicy;
|
|
|
-import javax.cache.expiry.Duration;
|
|
|
-import javax.cache.expiry.ExpiryPolicy;
|
|
|
-import javax.cache.spi.CachingProvider;
|
|
|
+import java.time.Duration;
|
|
|
import java.util.Map;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
public class FederationJCache extends FederationCache {
|
|
|
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(FederationJCache.class);
|
|
|
|
|
|
- private Cache<String, CacheRequest<String, ?>> cache;
|
|
|
+ private Cache<String, CacheRequest> cache;
|
|
|
|
|
|
private int cacheTimeToLive;
|
|
|
+ private long cacheEntityNums;
|
|
|
|
|
|
private boolean isCachingEnabled = false;
|
|
|
|
|
|
- private String className = this.getClass().getSimpleName();
|
|
|
+ private final String className = this.getClass().getSimpleName();
|
|
|
|
|
|
@Override
|
|
|
public boolean isCachingEnabled() {
|
|
@@ -64,33 +63,35 @@ public class FederationJCache extends FederationCache {
|
|
|
// no conflict or pick up a specific one in the future
|
|
|
cacheTimeToLive = pConf.getInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS,
|
|
|
YarnConfiguration.DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS);
|
|
|
+ cacheEntityNums = pConf.getLong(YarnConfiguration.FEDERATION_CACHE_ENTITY_NUMS,
|
|
|
+ YarnConfiguration.DEFAULT_FEDERATION_CACHE_ENTITY_NUMS);
|
|
|
if (cacheTimeToLive <= 0) {
|
|
|
isCachingEnabled = false;
|
|
|
return;
|
|
|
}
|
|
|
this.setStateStore(pStateStore);
|
|
|
- CachingProvider jcacheProvider = Caching.getCachingProvider();
|
|
|
- CacheManager jcacheManager = jcacheProvider.getCacheManager();
|
|
|
- this.cache = jcacheManager.getCache(className);
|
|
|
+ CacheManager cacheManager = CacheManagerBuilder.newCacheManagerBuilder().build(true);
|
|
|
+
|
|
|
if (this.cache == null) {
|
|
|
- LOG.info("Creating a JCache Manager with name {}.", className);
|
|
|
- Duration cacheExpiry = new Duration(TimeUnit.SECONDS, cacheTimeToLive);
|
|
|
- FactoryBuilder.SingletonFactory<ExpiryPolicy> expiryPolicySingletonFactory =
|
|
|
- new FactoryBuilder.SingletonFactory<>(new CreatedExpiryPolicy(cacheExpiry));
|
|
|
- MutableConfiguration<String, CacheRequest<String, ?>> configuration =
|
|
|
- new MutableConfiguration<>();
|
|
|
- configuration.setStoreByValue(false);
|
|
|
- configuration.setExpiryPolicyFactory(expiryPolicySingletonFactory);
|
|
|
- this.cache = jcacheManager.createCache(className, configuration);
|
|
|
+ LOG.info("Creating a JCache Manager with name {}. " +
|
|
|
+ "Cache TTL Time = {} secs. Cache Entity Nums = {}.", className, cacheTimeToLive,
|
|
|
+ cacheEntityNums);
|
|
|
+ // Set the number of caches
|
|
|
+ ResourcePoolsBuilder poolsBuilder = ResourcePoolsBuilder.heap(cacheEntityNums);
|
|
|
+ ExpiryPolicy expiryPolicy = ExpiryPolicyBuilder.timeToLiveExpiration(
|
|
|
+ Duration.ofSeconds(cacheTimeToLive));
|
|
|
+ CacheConfigurationBuilder<String, CacheRequest> configurationBuilder =
|
|
|
+ CacheConfigurationBuilder.newCacheConfigurationBuilder(
|
|
|
+ String.class, CacheRequest.class, poolsBuilder)
|
|
|
+ .withExpiry(expiryPolicy);
|
|
|
+ cache = cacheManager.createCache(className, configurationBuilder);
|
|
|
}
|
|
|
isCachingEnabled = true;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void clearCache() {
|
|
|
- CachingProvider jcacheProvider = Caching.getCachingProvider();
|
|
|
- CacheManager jcacheManager = jcacheProvider.getCacheManager();
|
|
|
- jcacheManager.destroyCache(className);
|
|
|
+
|
|
|
this.cache = null;
|
|
|
}
|
|
|
|
|
@@ -142,13 +143,12 @@ public class FederationJCache extends FederationCache {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- public Cache<String, CacheRequest<String, ?>> getCache() {
|
|
|
+ public Cache<String, CacheRequest> getCache() {
|
|
|
return cache;
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- public String getAppHomeSubClusterCacheKey(ApplicationId appId)
|
|
|
- throws YarnException {
|
|
|
+ public String getAppHomeSubClusterCacheKey(ApplicationId appId) {
|
|
|
return buildCacheKey(className, GET_APPLICATION_HOME_SUBCLUSTER_CACHEID,
|
|
|
appId.toString());
|
|
|
}
|