|
@@ -28,22 +28,10 @@ import java.util.concurrent.TimeUnit;
|
|
|
import java.util.Random;
|
|
|
import java.util.Collection;
|
|
|
|
|
|
-import javax.cache.Cache;
|
|
|
-import javax.cache.CacheManager;
|
|
|
-import javax.cache.Caching;
|
|
|
-import javax.cache.configuration.CompleteConfiguration;
|
|
|
-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.integration.CacheLoader;
|
|
|
import javax.cache.integration.CacheLoaderException;
|
|
|
-import javax.cache.spi.CachingProvider;
|
|
|
|
|
|
import org.apache.commons.collections.CollectionUtils;
|
|
|
import org.apache.commons.collections.MapUtils;
|
|
|
-import org.apache.commons.lang3.NotImplementedException;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.io.retry.RetryPolicies;
|
|
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
|
@@ -55,6 +43,8 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
+import org.apache.hadoop.yarn.server.federation.cache.FederationCache;
|
|
|
+import org.apache.hadoop.yarn.server.federation.cache.FederationJCache;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
|
|
|
import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
|
|
@@ -72,11 +62,9 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHome
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
|
|
|
-import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
|
|
|
-import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
|
|
@@ -101,6 +89,9 @@ import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
import com.zaxxer.hikari.pool.HikariPool.PoolInitializationException;
|
|
|
|
|
|
+import static org.apache.hadoop.yarn.server.federation.cache.FederationCache.buildPolicyConfigMap;
|
|
|
+import static org.apache.hadoop.yarn.server.federation.cache.FederationCache.buildSubClusterInfoMap;
|
|
|
+
|
|
|
/**
|
|
|
*
|
|
|
* The FederationStateStoreFacade is an utility wrapper that provides singleton
|
|
@@ -112,22 +103,15 @@ public final class FederationStateStoreFacade {
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(FederationStateStoreFacade.class);
|
|
|
|
|
|
- private static final String GET_SUBCLUSTERS_CACHEID = "getSubClusters";
|
|
|
- private static final String GET_POLICIES_CONFIGURATIONS_CACHEID =
|
|
|
- "getPoliciesConfigurations";
|
|
|
- private static final String GET_APPLICATION_HOME_SUBCLUSTER_CACHEID =
|
|
|
- "getApplicationHomeSubCluster";
|
|
|
-
|
|
|
private static final FederationStateStoreFacade FACADE =
|
|
|
new FederationStateStoreFacade();
|
|
|
|
|
|
private static Random rand = new Random(System.currentTimeMillis());
|
|
|
|
|
|
private FederationStateStore stateStore;
|
|
|
- private int cacheTimeToLive;
|
|
|
private Configuration conf;
|
|
|
- private Cache<Object, Object> cache;
|
|
|
private SubClusterResolver subclusterResolver;
|
|
|
+ private FederationCache federationCache;
|
|
|
|
|
|
private FederationStateStoreFacade() {
|
|
|
initializeFacadeInternal(new Configuration());
|
|
@@ -148,11 +132,11 @@ public final class FederationStateStoreFacade {
|
|
|
SubClusterResolver.class);
|
|
|
this.subclusterResolver.load();
|
|
|
|
|
|
- initCache();
|
|
|
+ federationCache = new FederationJCache();
|
|
|
+ federationCache.initCache(config, stateStore);
|
|
|
|
|
|
} catch (YarnException ex) {
|
|
|
- LOG.error("Failed to initialize the FederationStateStoreFacade object",
|
|
|
- ex);
|
|
|
+ LOG.error("Failed to initialize the FederationStateStoreFacade object", ex);
|
|
|
throw new RuntimeException(ex);
|
|
|
}
|
|
|
}
|
|
@@ -169,8 +153,8 @@ public final class FederationStateStoreFacade {
|
|
|
Configuration config) {
|
|
|
this.conf = config;
|
|
|
this.stateStore = store;
|
|
|
- clearCache();
|
|
|
- initCache();
|
|
|
+ federationCache.clearCache();
|
|
|
+ federationCache.initCache(config, stateStore);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -191,8 +175,7 @@ public final class FederationStateStoreFacade {
|
|
|
conf.getLong(YarnConfiguration.CLIENT_FAILOVER_SLEEPTIME_BASE_MS,
|
|
|
YarnConfiguration.DEFAULT_RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS),
|
|
|
TimeUnit.MILLISECONDS);
|
|
|
- Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
|
|
|
- new HashMap<Class<? extends Exception>, RetryPolicy>();
|
|
|
+ Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap = new HashMap<>();
|
|
|
exceptionToPolicyMap.put(FederationStateStoreRetriableException.class,
|
|
|
basePolicy);
|
|
|
exceptionToPolicyMap.put(CacheLoaderException.class, basePolicy);
|
|
@@ -203,47 +186,6 @@ public final class FederationStateStoreFacade {
|
|
|
return retryPolicy;
|
|
|
}
|
|
|
|
|
|
- private boolean isCachingEnabled() {
|
|
|
- return (cacheTimeToLive > 0);
|
|
|
- }
|
|
|
-
|
|
|
- private void initCache() {
|
|
|
- // Picking the JCache provider from classpath, need to make sure there's
|
|
|
- // no conflict or pick up a specific one in the future
|
|
|
- cacheTimeToLive =
|
|
|
- conf.getInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS,
|
|
|
- YarnConfiguration.DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS);
|
|
|
- if (isCachingEnabled()) {
|
|
|
- CachingProvider jcacheProvider = Caching.getCachingProvider();
|
|
|
- CacheManager jcacheManager = jcacheProvider.getCacheManager();
|
|
|
- this.cache = jcacheManager.getCache(this.getClass().getSimpleName());
|
|
|
- if (this.cache == null) {
|
|
|
- LOG.info("Creating a JCache Manager with name "
|
|
|
- + this.getClass().getSimpleName());
|
|
|
- Duration cacheExpiry = new Duration(TimeUnit.SECONDS, cacheTimeToLive);
|
|
|
- CompleteConfiguration<Object, Object> configuration =
|
|
|
- new MutableConfiguration<Object, Object>().setStoreByValue(false)
|
|
|
- .setReadThrough(true)
|
|
|
- .setExpiryPolicyFactory(
|
|
|
- new FactoryBuilder.SingletonFactory<ExpiryPolicy>(
|
|
|
- new CreatedExpiryPolicy(cacheExpiry)))
|
|
|
- .setCacheLoaderFactory(
|
|
|
- new FactoryBuilder.SingletonFactory<CacheLoader<Object, Object>>(
|
|
|
- new CacheLoaderImpl<Object, Object>()));
|
|
|
- this.cache = jcacheManager.createCache(this.getClass().getSimpleName(),
|
|
|
- configuration);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void clearCache() {
|
|
|
- CachingProvider jcacheProvider = Caching.getCachingProvider();
|
|
|
- CacheManager jcacheManager = jcacheProvider.getCacheManager();
|
|
|
-
|
|
|
- jcacheManager.destroyCache(this.getClass().getSimpleName());
|
|
|
- this.cache = null;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Returns the singleton instance of the FederationStateStoreFacade object.
|
|
|
*
|
|
@@ -263,7 +205,7 @@ public final class FederationStateStoreFacade {
|
|
|
*/
|
|
|
public SubClusterInfo getSubCluster(final SubClusterId subClusterId)
|
|
|
throws YarnException {
|
|
|
- if (isCachingEnabled()) {
|
|
|
+ if (federationCache.isCachingEnabled()) {
|
|
|
return getSubClusters(false).get(subClusterId);
|
|
|
} else {
|
|
|
GetSubClusterInfoResponse response = stateStore
|
|
@@ -287,10 +229,10 @@ public final class FederationStateStoreFacade {
|
|
|
*/
|
|
|
public SubClusterInfo getSubCluster(final SubClusterId subClusterId,
|
|
|
final boolean flushCache) throws YarnException {
|
|
|
- if (flushCache && isCachingEnabled()) {
|
|
|
+ if (flushCache && federationCache.isCachingEnabled()) {
|
|
|
LOG.info("Flushing subClusters from cache and rehydrating from store,"
|
|
|
+ " most likely on account of RM failover.");
|
|
|
- cache.remove(buildGetSubClustersCacheRequest(false));
|
|
|
+ federationCache.removeSubCluster(false);
|
|
|
}
|
|
|
return getSubCluster(subClusterId);
|
|
|
}
|
|
@@ -303,16 +245,15 @@ public final class FederationStateStoreFacade {
|
|
|
* @return the information of all active sub cluster(s)
|
|
|
* @throws YarnException if the call to the state store is unsuccessful
|
|
|
*/
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
- public Map<SubClusterId, SubClusterInfo> getSubClusters(
|
|
|
- final boolean filterInactiveSubClusters) throws YarnException {
|
|
|
+ public Map<SubClusterId, SubClusterInfo> getSubClusters(final boolean filterInactiveSubClusters)
|
|
|
+ throws YarnException {
|
|
|
try {
|
|
|
- if (isCachingEnabled()) {
|
|
|
- return (Map<SubClusterId, SubClusterInfo>) cache
|
|
|
- .get(buildGetSubClustersCacheRequest(filterInactiveSubClusters));
|
|
|
+ if (federationCache.isCachingEnabled()) {
|
|
|
+ return federationCache.getSubClusters(filterInactiveSubClusters);
|
|
|
} else {
|
|
|
- return buildSubClusterInfoMap(stateStore.getSubClusters(
|
|
|
- GetSubClustersInfoRequest.newInstance(filterInactiveSubClusters)));
|
|
|
+ GetSubClustersInfoRequest request =
|
|
|
+ GetSubClustersInfoRequest.newInstance(filterInactiveSubClusters);
|
|
|
+ return buildSubClusterInfoMap(stateStore.getSubClusters(request));
|
|
|
}
|
|
|
} catch (Throwable ex) {
|
|
|
throw new YarnException(ex);
|
|
@@ -327,15 +268,15 @@ public final class FederationStateStoreFacade {
|
|
|
* mapping for the queue
|
|
|
* @throws YarnException if the call to the state store is unsuccessful
|
|
|
*/
|
|
|
- public SubClusterPolicyConfiguration getPolicyConfiguration(
|
|
|
- final String queue) throws YarnException {
|
|
|
- if (isCachingEnabled()) {
|
|
|
+ public SubClusterPolicyConfiguration getPolicyConfiguration(final String queue)
|
|
|
+ throws YarnException {
|
|
|
+ if (federationCache.isCachingEnabled()) {
|
|
|
return getPoliciesConfigurations().get(queue);
|
|
|
} else {
|
|
|
-
|
|
|
+ GetSubClusterPolicyConfigurationRequest request =
|
|
|
+ GetSubClusterPolicyConfigurationRequest.newInstance(queue);
|
|
|
GetSubClusterPolicyConfigurationResponse response =
|
|
|
- stateStore.getPolicyConfiguration(
|
|
|
- GetSubClusterPolicyConfigurationRequest.newInstance(queue));
|
|
|
+ stateStore.getPolicyConfiguration(request);
|
|
|
if (response == null) {
|
|
|
return null;
|
|
|
} else {
|
|
@@ -352,16 +293,15 @@ public final class FederationStateStoreFacade {
|
|
|
* @return the policies for all currently active queues in the system
|
|
|
* @throws YarnException if the call to the state store is unsuccessful
|
|
|
*/
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
public Map<String, SubClusterPolicyConfiguration> getPoliciesConfigurations()
|
|
|
throws YarnException {
|
|
|
try {
|
|
|
- if (isCachingEnabled()) {
|
|
|
- return (Map<String, SubClusterPolicyConfiguration>) cache
|
|
|
- .get(buildGetPoliciesConfigurationsCacheRequest());
|
|
|
+ if (federationCache.isCachingEnabled()) {
|
|
|
+ return federationCache.getPoliciesConfigurations();
|
|
|
} else {
|
|
|
- return buildPolicyConfigMap(stateStore.getPoliciesConfigurations(
|
|
|
- GetSubClusterPoliciesConfigurationsRequest.newInstance()));
|
|
|
+ GetSubClusterPoliciesConfigurationsRequest request =
|
|
|
+ GetSubClusterPoliciesConfigurationsRequest.newInstance();
|
|
|
+ return buildPolicyConfigMap(stateStore.getPoliciesConfigurations(request));
|
|
|
}
|
|
|
} catch (Throwable ex) {
|
|
|
throw new YarnException(ex);
|
|
@@ -396,7 +336,6 @@ public final class FederationStateStoreFacade {
|
|
|
ApplicationHomeSubCluster appHomeSubCluster) throws YarnException {
|
|
|
stateStore.updateApplicationHomeSubCluster(
|
|
|
UpdateApplicationHomeSubClusterRequest.newInstance(appHomeSubCluster));
|
|
|
- return;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -410,10 +349,8 @@ public final class FederationStateStoreFacade {
|
|
|
public SubClusterId getApplicationHomeSubCluster(ApplicationId appId)
|
|
|
throws YarnException {
|
|
|
try {
|
|
|
- if (isCachingEnabled()) {
|
|
|
- SubClusterId value = SubClusterId.class.cast(
|
|
|
- cache.get(buildGetApplicationHomeSubClusterRequest(appId)));
|
|
|
- return value;
|
|
|
+ if (federationCache.isCachingEnabled()) {
|
|
|
+ return federationCache.getApplicationHomeSubCluster(appId);
|
|
|
} else {
|
|
|
GetApplicationHomeSubClusterResponse response = stateStore.getApplicationHomeSubCluster(
|
|
|
GetApplicationHomeSubClusterRequest.newInstance(appId));
|
|
@@ -550,196 +487,6 @@ public final class FederationStateStoreFacade {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private Map<SubClusterId, SubClusterInfo> buildSubClusterInfoMap(
|
|
|
- final GetSubClustersInfoResponse response) {
|
|
|
- List<SubClusterInfo> subClusters = response.getSubClusters();
|
|
|
- Map<SubClusterId, SubClusterInfo> subClustersMap =
|
|
|
- new HashMap<>(subClusters.size());
|
|
|
- for (SubClusterInfo subCluster : subClusters) {
|
|
|
- subClustersMap.put(subCluster.getSubClusterId(), subCluster);
|
|
|
- }
|
|
|
- return subClustersMap;
|
|
|
- }
|
|
|
-
|
|
|
- private Object buildGetSubClustersCacheRequest(
|
|
|
- final boolean filterInactiveSubClusters) {
|
|
|
- final String cacheKey =
|
|
|
- buildCacheKey(getClass().getSimpleName(), GET_SUBCLUSTERS_CACHEID,
|
|
|
- Boolean.toString(filterInactiveSubClusters));
|
|
|
- CacheRequest<String, Map<SubClusterId, SubClusterInfo>> cacheRequest =
|
|
|
- new CacheRequest<String, Map<SubClusterId, SubClusterInfo>>(cacheKey,
|
|
|
- new Func<String, Map<SubClusterId, SubClusterInfo>>() {
|
|
|
- @Override
|
|
|
- public Map<SubClusterId, SubClusterInfo> invoke(String key)
|
|
|
- throws Exception {
|
|
|
- GetSubClustersInfoResponse subClusters =
|
|
|
- stateStore.getSubClusters(GetSubClustersInfoRequest
|
|
|
- .newInstance(filterInactiveSubClusters));
|
|
|
- return buildSubClusterInfoMap(subClusters);
|
|
|
- }
|
|
|
- });
|
|
|
- return cacheRequest;
|
|
|
- }
|
|
|
-
|
|
|
- private Map<String, SubClusterPolicyConfiguration> buildPolicyConfigMap(
|
|
|
- GetSubClusterPoliciesConfigurationsResponse response) {
|
|
|
- List<SubClusterPolicyConfiguration> policyConfigs =
|
|
|
- response.getPoliciesConfigs();
|
|
|
- Map<String, SubClusterPolicyConfiguration> queuePolicyConfigs =
|
|
|
- new HashMap<>();
|
|
|
- for (SubClusterPolicyConfiguration policyConfig : policyConfigs) {
|
|
|
- queuePolicyConfigs.put(policyConfig.getQueue(), policyConfig);
|
|
|
- }
|
|
|
- return queuePolicyConfigs;
|
|
|
- }
|
|
|
-
|
|
|
- private Object buildGetPoliciesConfigurationsCacheRequest() {
|
|
|
- final String cacheKey = buildCacheKey(getClass().getSimpleName(),
|
|
|
- GET_POLICIES_CONFIGURATIONS_CACHEID, null);
|
|
|
- CacheRequest<String, Map<String, SubClusterPolicyConfiguration>> cacheRequest =
|
|
|
- new CacheRequest<String, Map<String, SubClusterPolicyConfiguration>>(
|
|
|
- cacheKey,
|
|
|
- new Func<String, Map<String, SubClusterPolicyConfiguration>>() {
|
|
|
- @Override
|
|
|
- public Map<String, SubClusterPolicyConfiguration> invoke(
|
|
|
- String key) throws Exception {
|
|
|
- GetSubClusterPoliciesConfigurationsResponse policyConfigs =
|
|
|
- stateStore.getPoliciesConfigurations(
|
|
|
- GetSubClusterPoliciesConfigurationsRequest
|
|
|
- .newInstance());
|
|
|
- return buildPolicyConfigMap(policyConfigs);
|
|
|
- }
|
|
|
- });
|
|
|
- return cacheRequest;
|
|
|
- }
|
|
|
-
|
|
|
- private Object buildGetApplicationHomeSubClusterRequest(ApplicationId applicationId) {
|
|
|
- final String cacheKey = buildCacheKey(getClass().getSimpleName(),
|
|
|
- GET_APPLICATION_HOME_SUBCLUSTER_CACHEID, applicationId.toString());
|
|
|
- CacheRequest<String, SubClusterId> cacheRequest = new CacheRequest<>(
|
|
|
- cacheKey,
|
|
|
- input -> {
|
|
|
-
|
|
|
- GetApplicationHomeSubClusterRequest request =
|
|
|
- GetApplicationHomeSubClusterRequest.newInstance(applicationId);
|
|
|
- GetApplicationHomeSubClusterResponse response =
|
|
|
- stateStore.getApplicationHomeSubCluster(request);
|
|
|
-
|
|
|
- ApplicationHomeSubCluster appHomeSubCluster = response.getApplicationHomeSubCluster();
|
|
|
- SubClusterId subClusterId = appHomeSubCluster.getHomeSubCluster();
|
|
|
-
|
|
|
- return subClusterId;
|
|
|
- });
|
|
|
- return cacheRequest;
|
|
|
- }
|
|
|
-
|
|
|
- protected String buildCacheKey(String typeName, String methodName,
|
|
|
- String argName) {
|
|
|
- StringBuilder buffer = new StringBuilder();
|
|
|
- buffer.append(typeName).append(".")
|
|
|
- .append(methodName);
|
|
|
- if (argName != null) {
|
|
|
- buffer.append("::");
|
|
|
- buffer.append(argName);
|
|
|
- }
|
|
|
- return buffer.toString();
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Internal class that implements the CacheLoader interface that can be
|
|
|
- * plugged into the CacheManager to load objects into the cache for specified
|
|
|
- * keys.
|
|
|
- */
|
|
|
- private static class CacheLoaderImpl<K, V> implements CacheLoader<K, V> {
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
- @Override
|
|
|
- public V load(K key) throws CacheLoaderException {
|
|
|
- try {
|
|
|
- CacheRequest<K, V> query = (CacheRequest<K, V>) key;
|
|
|
- assert query != null;
|
|
|
- return query.getValue();
|
|
|
- } catch (Throwable ex) {
|
|
|
- throw new CacheLoaderException(ex);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public Map<K, V> loadAll(Iterable<? extends K> keys)
|
|
|
- throws CacheLoaderException {
|
|
|
- // The FACADE does not use the Cache's getAll API. Hence this is not
|
|
|
- // required to be implemented
|
|
|
- throw new NotImplementedException("Code is not implemented");
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Internal class that encapsulates the cache key and a function that returns
|
|
|
- * the value for the specified key.
|
|
|
- */
|
|
|
- private static class CacheRequest<K, V> {
|
|
|
- private K key;
|
|
|
- private Func<K, V> func;
|
|
|
-
|
|
|
- CacheRequest(K key, Func<K, V> func) {
|
|
|
- this.key = key;
|
|
|
- this.func = func;
|
|
|
- }
|
|
|
-
|
|
|
- public V getValue() throws Exception {
|
|
|
- return func.invoke(key);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int hashCode() {
|
|
|
- final int prime = 31;
|
|
|
- int result = 1;
|
|
|
- result = prime * result + ((key == null) ? 0 : key.hashCode());
|
|
|
- return result;
|
|
|
- }
|
|
|
-
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
- @Override
|
|
|
- public boolean equals(Object obj) {
|
|
|
- if (this == obj) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- if (obj == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- if (getClass() != obj.getClass()) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- CacheRequest<K, V> other = (CacheRequest<K, V>) obj;
|
|
|
- if (key == null) {
|
|
|
- if (other.key != null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- } else if (!key.equals(other.key)) {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- return true;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Encapsulates a method that has one parameter and returns a value of the
|
|
|
- * type specified by the TResult parameter.
|
|
|
- */
|
|
|
- protected interface Func<T, TResult> {
|
|
|
- TResult invoke(T input) throws Exception;
|
|
|
- }
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- public Cache<Object, Object> getCache() {
|
|
|
- return cache;
|
|
|
- }
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- protected Object getAppHomeSubClusterCacheRequest(ApplicationId applicationId) {
|
|
|
- return buildGetApplicationHomeSubClusterRequest(applicationId);
|
|
|
- }
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
public FederationStateStore getStateStore() {
|
|
|
return stateStore;
|
|
@@ -1232,4 +979,9 @@ public final class FederationStateStoreFacade {
|
|
|
return null;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public FederationCache getFederationCache() {
|
|
|
+ return federationCache;
|
|
|
+ }
|
|
|
}
|