You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2022/06/01 21:50:08 UTC
[accumulo] branch main updated: Clear property caches on RPC request for props (#2740)
This is an automated email from the ASF dual-hosted git repository.
ctubbsii pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/main by this push:
new 48165863bf Clear property caches on RPC request for props (#2740)
48165863bf is described below
commit 48165863bf359b2e0ddf4cddbcc6f33f7ffe4ce7
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Wed Jun 1 17:50:02 2022 -0400
Clear property caches on RPC request for props (#2740)
Mitigation for test instability in #2739
* Clear namespace, table, and system property cache entries when that
particular configuration is being requested from the client
API improvements for PropStore
* Rename PropCacheKey to PropStoreKey, since it's a first-class citizen
of the PropStore interface
* Add getCache() method to PropStore, because it is expected that
PropStores are caching, so this makes it explicit
* Use .getCache().remove() instead of casting to ZooPropStore and
simulating a ZK change event to remove an item from the cache to force
it to reload
---
.../server/client/ClientServiceHandler.java | 6 ++
.../server/conf/NamespaceConfiguration.java | 6 +-
.../accumulo/server/conf/SystemConfiguration.java | 4 +-
.../server/conf/ZooBasedConfiguration.java | 22 ++--
.../server/conf/store/NamespacePropKey.java | 2 +-
.../accumulo/server/conf/store/PropCache.java | 7 +-
.../server/conf/store/PropChangeListener.java | 6 +-
.../accumulo/server/conf/store/PropStore.java | 32 +++---
.../store/{PropCacheKey.java => PropStoreKey.java} | 16 +--
.../accumulo/server/conf/store/SystemPropKey.java | 2 +-
.../accumulo/server/conf/store/TablePropKey.java | 2 +-
.../conf/store/impl/PropCacheCaffeineImpl.java | 31 +++---
.../server/conf/store/impl/PropStoreEventTask.java | 30 +++---
.../server/conf/store/impl/PropStoreWatcher.java | 62 +++++------
.../server/conf/store/impl/ZooPropLoader.java | 38 +++----
.../server/conf/store/impl/ZooPropStore.java | 114 +++++++++++----------
.../server/conf/util/ConfigTransformer.java | 78 +++++++-------
.../accumulo/server/conf/util/PropSnapshot.java | 30 +++---
.../accumulo/server/conf/util/TransformToken.java | 8 +-
.../server/conf/NamespaceConfigurationTest.java | 6 +-
...PropCacheKeyTest.java => PropStoreKeyTest.java} | 16 +--
.../server/conf/store/impl/PropStoreEventTest.java | 8 +-
.../server/conf/store/impl/ZooPropLoaderTest.java | 100 +++++++++---------
.../server/conf/store/impl/ZooPropStoreTest.java | 66 ++++++------
.../test/conf/store/PropCacheCaffeineImplZkIT.java | 6 +-
.../test/conf/store/PropStoreZooKeeperIT.java | 22 ++--
.../accumulo/test/conf/store/ZooBasedConfigIT.java | 14 +--
27 files changed, 376 insertions(+), 358 deletions(-)
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 440cac12ce..5ca421a5ba 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -60,6 +60,9 @@ import org.apache.accumulo.core.security.TablePermission;
import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
import org.apache.accumulo.core.trace.thrift.TInfo;
import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
import org.apache.accumulo.server.security.SecurityOperation;
import org.apache.accumulo.server.util.ServerBulkImportStatus;
import org.apache.accumulo.server.util.TableDiskUsage;
@@ -305,6 +308,7 @@ public class ClientServiceHandler implements ClientService.Iface {
ConfigurationType type) throws TException {
switch (type) {
case CURRENT:
+ context.getPropStore().getCache().remove(SystemPropKey.of(context));
return conf(credentials, context.getConfiguration());
case SITE:
return conf(credentials, context.getSiteConfiguration());
@@ -318,6 +322,7 @@ public class ClientServiceHandler implements ClientService.Iface {
public Map<String,String> getTableConfiguration(TInfo tinfo, TCredentials credentials,
String tableName) throws TException, ThriftTableOperationException {
TableId tableId = checkTableId(context, tableName, null);
+ context.getPropStore().getCache().remove(TablePropKey.of(context, tableId));
AccumuloConfiguration config = context.getTableConfiguration(tableId);
return conf(credentials, config);
}
@@ -458,6 +463,7 @@ public class ClientServiceHandler implements ClientService.Iface {
throw new ThriftTableOperationException(null, ns, null,
TableOperationExceptionType.NAMESPACE_NOTFOUND, why);
}
+ context.getPropStore().getCache().remove(NamespacePropKey.of(context, namespaceId));
AccumuloConfiguration config = context.getNamespaceConfiguration(namespaceId);
return conf(credentials, config);
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
index 2d8b19d615..e60ff220ab 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -45,7 +45,7 @@ public class NamespaceConfiguration extends ZooBasedConfiguration {
String key = property.getKey();
- var namespaceId = getPropCacheKey().getId();
+ var namespaceId = getPropStoreKey().getId();
if (namespaceId != null && namespaceId.equals(Namespace.ACCUMULO.id())
&& isIteratorOrConstraint(key)) {
// ignore iterators from parent if system namespace
@@ -81,10 +81,10 @@ public class NamespaceConfiguration extends ZooBasedConfiguration {
}
protected NamespaceId getNamespaceId() {
- NamespaceId id = (NamespaceId) getPropCacheKey().getId();
+ NamespaceId id = (NamespaceId) getPropStoreKey().getId();
if (id == null) {
throw new IllegalArgumentException(
- "Invalid request for namespace id on " + getPropCacheKey());
+ "Invalid request for namespace id on " + getPropStoreKey());
}
return id;
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/SystemConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/SystemConfiguration.java
index 661f3fbcae..9e2429e218 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/SystemConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/SystemConfiguration.java
@@ -31,9 +31,9 @@ public class SystemConfiguration extends ZooBasedConfiguration {
private final RuntimeFixedProperties runtimeFixedProps;
- public SystemConfiguration(ServerContext context, SystemPropKey propCacheKey,
+ public SystemConfiguration(ServerContext context, SystemPropKey propStoreKey,
AccumuloConfiguration parent) {
- super(log, context, propCacheKey, parent);
+ super(log, context, propStoreKey, parent);
runtimeFixedProps = new RuntimeFixedProperties(getSnapshot(), context.getSiteConfiguration());
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooBasedConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooBasedConfiguration.java
index 2a847af26b..9f5c57719b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooBasedConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooBasedConfiguration.java
@@ -26,7 +26,7 @@ import java.util.function.Predicate;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.accumulo.server.conf.util.PropSnapshot;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
@@ -49,23 +49,23 @@ public class ZooBasedConfiguration extends AccumuloConfiguration {
protected final Logger log;
private final AccumuloConfiguration parent;
- private final PropCacheKey<?> propCacheKey;
+ private final PropStoreKey<?> propStoreKey;
private final PropSnapshot propSnapshot;
- public ZooBasedConfiguration(Logger log, ServerContext context, PropCacheKey<?> propCacheKey,
+ public ZooBasedConfiguration(Logger log, ServerContext context, PropStoreKey<?> propStoreKey,
AccumuloConfiguration parent) {
this.log = requireNonNull(log, "a Logger must be supplied");
requireNonNull(context, "the context cannot be null");
- this.propCacheKey = requireNonNull(propCacheKey, "a PropCacheId must be supplied");
+ this.propStoreKey = requireNonNull(propStoreKey, "a PropCacheId must be supplied");
this.parent = requireNonNull(parent, "An AccumuloConfiguration parent must be supplied");
- propSnapshot = PropSnapshot.create(propCacheKey, context.getPropStore());
+ propSnapshot = PropSnapshot.create(propStoreKey, context.getPropStore());
}
@VisibleForTesting
- public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
- propSnapshot.zkChangeEvent(propCacheKey);
+ public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
+ propSnapshot.zkChangeEvent(propStoreKey);
}
public long getDataVersion() {
@@ -93,7 +93,7 @@ public class ZooBasedConfiguration extends AccumuloConfiguration {
count += dataVersion;
}
- log.trace("update count result for: {} - data version: {} update: {}", propCacheKey,
+ log.trace("update count result for: {} - data version: {} update: {}", propStoreKey,
dataVersion, count);
return count;
}
@@ -103,8 +103,8 @@ public class ZooBasedConfiguration extends AccumuloConfiguration {
return parent;
}
- public PropCacheKey<?> getPropCacheKey() {
- return propCacheKey;
+ public PropStoreKey<?> getPropStoreKey() {
+ return propStoreKey;
}
@Override
@@ -128,7 +128,7 @@ public class ZooBasedConfiguration extends AccumuloConfiguration {
Map<String,String> theseProps = getSnapshot();
- log.trace("getProperties() for: {} filter: {}, have: {}, passed: {}", getPropCacheKey(), filter,
+ log.trace("getProperties() for: {} filter: {}, have: {}, passed: {}", getPropStoreKey(), filter,
theseProps, props);
for (Map.Entry<String,String> p : theseProps.entrySet()) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/NamespacePropKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/NamespacePropKey.java
index b13fb4d8b8..49fd279a02 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/NamespacePropKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/NamespacePropKey.java
@@ -28,7 +28,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
import org.apache.accumulo.server.ServerContext;
import org.checkerframework.checker.nullness.qual.NonNull;
-public class NamespacePropKey extends PropCacheKey<NamespaceId> {
+public class NamespacePropKey extends PropStoreKey<NamespaceId> {
private NamespacePropKey(final InstanceId instanceId, final String path, final NamespaceId id) {
super(instanceId, path, id);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCache.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCache.java
index 9d50189a17..9099e7fb85 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCache.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCache.java
@@ -24,13 +24,10 @@ import org.checkerframework.checker.nullness.qual.Nullable;
public interface PropCache {
@Nullable
- VersionedProperties get(final PropCacheKey<?> propCacheKey);
+ VersionedProperties get(final PropStoreKey<?> propStoreKey);
- void remove(final PropCacheKey<?> propCacheKey);
+ void remove(final PropStoreKey<?> propStoreKey);
void removeAll();
- @Nullable
- VersionedProperties getWithoutCaching(PropCacheKey<?> propCacheKey);
-
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropChangeListener.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropChangeListener.java
index 4e4b010cf2..409629951d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropChangeListener.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropChangeListener.java
@@ -23,18 +23,18 @@ public interface PropChangeListener {
/**
* Signal that a ZooKeeper data change event occurred and that the data has changed.
*/
- void zkChangeEvent(final PropCacheKey<?> propCacheKey);
+ void zkChangeEvent(final PropStoreKey<?> propStoreKey);
/**
* Signal that a cache change event occurred - cache change events occur on eviction or
* invalidation of the cache entry. The underlying data may or may not have changed.
*/
- void cacheChangeEvent(final PropCacheKey<?> propCacheKey);
+ void cacheChangeEvent(final PropStoreKey<?> propStoreKey);
/**
* Signal that the node had been deleted from ZooKeeper.
*/
- void deleteEvent(final PropCacheKey<?> propCacheKey);
+ void deleteEvent(final PropStoreKey<?> propStoreKey);
/**
* A ZooKeeper connection event (session closed, expired...) and that
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStore.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStore.java
index 8315c22c33..582cafb5b9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStore.java
@@ -23,32 +23,33 @@ import java.util.Map;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
public interface PropStore {
/**
* Test that a node for properties exists without throwing a KeeperException.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @return true if the property node exists, false otherwise.
* @throws IllegalStateException
* if the check fails due to interrupt.
*/
- boolean exists(PropCacheKey<?> propCacheKey);
+ boolean exists(PropStoreKey<?> propStoreKey);
/**
* Create an initial entry for the PropCacheId. If properties already exist, they are not
* modified.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @param props
* a map of property k,v pairs
* @throws IllegalStateException
* if the updates fails because of an underlying store exception
*/
- void create(PropCacheKey<?> propCacheKey, Map<String,String> props);
+ void create(PropStoreKey<?> propStoreKey, Map<String,String> props);
/**
*
@@ -60,42 +61,42 @@ public interface PropStore {
* not exist for the propCacheId
*/
@NonNull
- VersionedProperties get(PropCacheKey<?> propCacheId);
+ VersionedProperties get(PropStoreKey<?> propCacheId);
/**
* Adds or updates current properties. If the property currently exists it is overwritten,
* otherwise it is added.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @param props
* a map of property k,v pairs
* @throws IllegalStateException
* if the values cannot be written or if an underlying store exception occurs.
*/
- void putAll(PropCacheKey<?> propCacheKey, Map<String,String> props);
+ void putAll(PropStoreKey<?> propStoreKey, Map<String,String> props);
/**
* Delete the store node from the underlying store.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @throws IllegalStateException
* if the updates fails because of an underlying store exception
*/
- void delete(PropCacheKey<?> propCacheKey);
+ void delete(PropStoreKey<?> propStoreKey);
/**
* Deletes individual properties specified by the set of keys.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @param keys
* a set of keys.
* @throws IllegalStateException
* if the values cannot be deleted or if an underlying store exception occurs.
*/
- void removeProperties(PropCacheKey<?> propCacheKey, Collection<String> keys);
+ void removeProperties(PropStoreKey<?> propStoreKey, Collection<String> keys);
/**
* External processes can register for notifications if the properties change. Normally processes
@@ -107,11 +108,16 @@ public interface PropStore {
* but listeners should not perform lengthy operations on the notification to prevent delaying
* other listeners from receive timely notification of the changes detected.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @param listener
* a listener
*/
- void registerAsListener(PropCacheKey<?> propCacheKey, PropChangeListener listener);
+ void registerAsListener(PropStoreKey<?> propStoreKey, PropChangeListener listener);
+
+ PropCache getCache();
+
+ @Nullable
+ VersionedProperties getWithoutCaching(PropStoreKey<?> propStoreKey);
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCacheKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
similarity index 90%
rename from server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCacheKey.java
rename to server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
index 171a3f8274..c3fe88c9b8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropCacheKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/PropStoreKey.java
@@ -41,12 +41,12 @@ import org.slf4j.LoggerFactory;
* Provides utility methods from constructing different id based on type and methods to parse a
* ZooKeeper path and return a prop cache id.
*/
-public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
- implements Comparable<PropCacheKey<ID_TYPE>> {
+public abstract class PropStoreKey<ID_TYPE extends AbstractId<ID_TYPE>>
+ implements Comparable<PropStoreKey<ID_TYPE>> {
public static final String PROP_NODE_NAME = "encoded_props";
- private static final Logger log = LoggerFactory.getLogger(PropCacheKey.class);
+ private static final Logger log = LoggerFactory.getLogger(PropStoreKey.class);
// indices for path.split();
public static final int TYPE_TOKEN_POSITION = 3;
@@ -62,7 +62,7 @@ public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
private final String path;
- protected PropCacheKey(final InstanceId instanceId, final String path, final ID_TYPE id) {
+ protected PropStoreKey(final InstanceId instanceId, final String path, final ID_TYPE id) {
this.instanceId = instanceId;
this.path = path;
this.id = id;
@@ -87,7 +87,7 @@ public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
* the path
* @return the prop cache id
*/
- public static @Nullable PropCacheKey<?> fromPath(final String path) {
+ public static @Nullable PropStoreKey<?> fromPath(final String path) {
String[] tokens = path.split("/");
InstanceId instanceId;
@@ -111,8 +111,8 @@ public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
}
@Override
- public int compareTo(@NonNull PropCacheKey<ID_TYPE> other) {
- return Comparator.comparing(PropCacheKey<ID_TYPE>::getPath).compare(this, other);
+ public int compareTo(@NonNull PropStoreKey<ID_TYPE> other) {
+ return Comparator.comparing(PropStoreKey<ID_TYPE>::getPath).compare(this, other);
}
@Override
@@ -121,7 +121,7 @@ public abstract class PropCacheKey<ID_TYPE extends AbstractId<ID_TYPE>>
return true;
if (o == null || getClass() != o.getClass())
return false;
- PropCacheKey<?> that = (PropCacheKey<?>) o;
+ PropStoreKey<?> that = (PropStoreKey<?>) o;
if (getId().getClass() != that.getId().getClass()) {
return false;
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/SystemPropKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/SystemPropKey.java
index f960f4a242..369b0a4a8b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/SystemPropKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/SystemPropKey.java
@@ -25,7 +25,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
import org.apache.accumulo.server.ServerContext;
import org.checkerframework.checker.nullness.qual.NonNull;
-public class SystemPropKey extends PropCacheKey<InstanceId> {
+public class SystemPropKey extends PropStoreKey<InstanceId> {
private SystemPropKey(final InstanceId instanceId, final String path) {
super(instanceId, path, instanceId);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/TablePropKey.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/TablePropKey.java
index d8ee686ca3..061725401f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/TablePropKey.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/TablePropKey.java
@@ -27,7 +27,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
import org.apache.accumulo.server.ServerContext;
import org.checkerframework.checker.nullness.qual.NonNull;
-public class TablePropKey extends PropCacheKey<TableId> {
+public class TablePropKey extends PropStoreKey<TableId> {
public static TablePropKey of(final ServerContext context, final TableId tableId) {
return of(context.getInstanceID(), tableId);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropCacheCaffeineImpl.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropCacheCaffeineImpl.java
index 6433a51a4c..e5c84b65cb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropCacheCaffeineImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropCacheCaffeineImpl.java
@@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.accumulo.core.util.threads.ThreadPools;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
import org.apache.accumulo.server.conf.store.PropCache;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -47,9 +47,9 @@ public class PropCacheCaffeineImpl implements PropCache {
private final PropStoreMetrics metrics;
- private final LoadingCache<PropCacheKey<?>,VersionedProperties> cache;
+ private final LoadingCache<PropStoreKey<?>,VersionedProperties> cache;
- private PropCacheCaffeineImpl(final CacheLoader<PropCacheKey<?>,VersionedProperties> cacheLoader,
+ private PropCacheCaffeineImpl(final CacheLoader<PropStoreKey<?>,VersionedProperties> cacheLoader,
final PropStoreMetrics metrics, final Ticker ticker, boolean runTasksInline) {
this.metrics = metrics;
var builder = Caffeine.newBuilder().refreshAfterWrite(REFRESH_MIN, BASE_TIME_UNITS)
@@ -69,28 +69,28 @@ public class PropCacheCaffeineImpl implements PropCache {
return metrics;
}
- void evictionNotifier(PropCacheKey<?> propCacheKey, VersionedProperties value,
+ void evictionNotifier(PropStoreKey<?> propStoreKey, VersionedProperties value,
RemovalCause cause) {
- log.trace("Evicted: ID: {} was evicted from cache. Reason: {}", propCacheKey, cause);
+ log.trace("Evicted: ID: {} was evicted from cache. Reason: {}", propStoreKey, cause);
metrics.incrEviction();
}
@Override
- public @Nullable VersionedProperties get(PropCacheKey<?> propCacheKey) {
- log.trace("Called get() for {}", propCacheKey);
+ public @Nullable VersionedProperties get(PropStoreKey<?> propStoreKey) {
+ log.trace("Called get() for {}", propStoreKey);
try {
- return cache.get(propCacheKey);
+ return cache.get(propStoreKey);
} catch (Exception ex) {
- log.info("Cache failed to retrieve properties for: " + propCacheKey, ex);
+ log.info("Cache failed to retrieve properties for: " + propStoreKey, ex);
metrics.incrZkError();
return null;
}
}
@Override
- public void remove(PropCacheKey<?> propCacheKey) {
- log.trace("clear {} from cache", propCacheKey);
- cache.invalidate(propCacheKey);
+ public void remove(PropStoreKey<?> propStoreKey) {
+ log.trace("clear {} from cache", propStoreKey);
+ cache.invalidate(propStoreKey);
}
@Override
@@ -105,13 +105,12 @@ public class PropCacheCaffeineImpl implements PropCache {
* processing so there is no reason to store them in the cache at this time. If they are used, a
* normal cache get will load the property into the cache.
*
- * @param propCacheKey
+ * @param propStoreKey
* the property id
* @return the version properties if cached, otherwise return null.
*/
- @Override
- public @Nullable VersionedProperties getWithoutCaching(PropCacheKey<?> propCacheKey) {
- return cache.getIfPresent(propCacheKey);
+ public @Nullable VersionedProperties getWithoutCaching(PropStoreKey<?> propStoreKey) {
+ return cache.getIfPresent(propStoreKey);
}
public static class Builder {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTask.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTask.java
index c87e4e8807..182d50cccc 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTask.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTask.java
@@ -20,8 +20,8 @@ package org.apache.accumulo.server.conf.store.impl;
import java.util.Set;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
import org.apache.accumulo.server.conf.store.PropChangeListener;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
/**
* Provides a simple runnable base task for notifying listeners for PropStore event change
@@ -29,7 +29,7 @@ import org.apache.accumulo.server.conf.store.PropChangeListener;
*/
public abstract class PropStoreEventTask implements Runnable {
- private final PropCacheKey<?> propCacheKey;
+ private final PropStoreKey<?> propStoreKey;
private final Set<PropChangeListener> listeners;
/**
@@ -39,60 +39,60 @@ public abstract class PropStoreEventTask implements Runnable {
* the set of listeners.
*/
private PropStoreEventTask(final Set<PropChangeListener> listeners) {
- this.propCacheKey = null;
+ this.propStoreKey = null;
this.listeners = listeners;
}
/**
* Used when listeners for the specified prop cahe key should receive a notification.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @param listeners
* the set of listeners
*/
- private PropStoreEventTask(final PropCacheKey<?> propCacheKey,
+ private PropStoreEventTask(final PropStoreKey<?> propStoreKey,
final Set<PropChangeListener> listeners) {
- this.propCacheKey = propCacheKey;
+ this.propStoreKey = propStoreKey;
this.listeners = listeners;
}
public static class PropStoreZkChangeEventTask extends PropStoreEventTask {
- PropStoreZkChangeEventTask(final PropCacheKey<?> propCacheKey,
+ PropStoreZkChangeEventTask(final PropStoreKey<?> propStoreKey,
final Set<PropChangeListener> listeners) {
- super(propCacheKey, listeners);
+ super(propStoreKey, listeners);
}
@Override
public void run() {
- super.listeners.forEach(listener -> listener.zkChangeEvent(super.propCacheKey));
+ super.listeners.forEach(listener -> listener.zkChangeEvent(super.propStoreKey));
}
}
public static class PropStoreCacheChangeEventTask extends PropStoreEventTask {
- PropStoreCacheChangeEventTask(final PropCacheKey<?> propCacheKey,
+ PropStoreCacheChangeEventTask(final PropStoreKey<?> propStoreKey,
final Set<PropChangeListener> listeners) {
- super(propCacheKey, listeners);
+ super(propStoreKey, listeners);
}
@Override
public void run() {
- super.listeners.forEach(listener -> listener.cacheChangeEvent(super.propCacheKey));
+ super.listeners.forEach(listener -> listener.cacheChangeEvent(super.propStoreKey));
}
}
public static class PropStoreDeleteEventTask extends PropStoreEventTask {
- PropStoreDeleteEventTask(final PropCacheKey<?> propCacheKey,
+ PropStoreDeleteEventTask(final PropStoreKey<?> propStoreKey,
final Set<PropChangeListener> listeners) {
- super(propCacheKey, listeners);
+ super(propStoreKey, listeners);
}
@Override
public void run() {
- super.listeners.forEach(listener -> listener.deleteEvent(super.propCacheKey));
+ super.listeners.forEach(listener -> listener.deleteEvent(super.propStoreKey));
}
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreWatcher.java
index 5e8a6e7ab0..dd84d17317 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreWatcher.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/PropStoreWatcher.java
@@ -30,8 +30,8 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
import org.apache.accumulo.server.conf.store.PropChangeListener;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.checkerframework.checker.nullness.qual.NonNull;
@@ -65,7 +65,7 @@ public class PropStoreWatcher implements Watcher {
private final ReentrantReadWriteLock.WriteLock listenerWriteLock = listenerLock.writeLock();
// access should be guarded by acquiring the listener read or write lock
- private final Map<PropCacheKey<?>,Set<PropChangeListener>> listeners = new HashMap<>();
+ private final Map<PropStoreKey<?>,Set<PropChangeListener>> listeners = new HashMap<>();
private final ReadyMonitor zkReadyMonitor;
@@ -73,11 +73,11 @@ public class PropStoreWatcher implements Watcher {
this.zkReadyMonitor = zkReadyMonitor;
}
- public void registerListener(final PropCacheKey<?> propCacheKey,
+ public void registerListener(final PropStoreKey<?> propStoreKey,
final PropChangeListener listener) {
listenerWriteLock.lock();
try {
- Set<PropChangeListener> set = listeners.computeIfAbsent(propCacheKey, s -> new HashSet<>());
+ Set<PropChangeListener> set = listeners.computeIfAbsent(propStoreKey, s -> new HashSet<>());
set.add(listener);
} finally {
listenerWriteLock.unlock();
@@ -97,28 +97,28 @@ public class PropStoreWatcher implements Watcher {
public void process(final WatchedEvent event) {
String path;
- PropCacheKey<?> propCacheKey;
+ PropStoreKey<?> propStoreKey;
switch (event.getType()) {
case NodeDataChanged:
path = event.getPath();
log.trace("handle change event for path: {}", path);
- propCacheKey = PropCacheKey.fromPath(path);
- if (propCacheKey != null) {
- signalZkChangeEvent(propCacheKey);
+ propStoreKey = PropStoreKey.fromPath(path);
+ if (propStoreKey != null) {
+ signalZkChangeEvent(propStoreKey);
}
break;
case NodeDeleted:
path = event.getPath();
log.trace("handle delete event for path: {}", path);
- propCacheKey = PropCacheKey.fromPath(path);
- if (propCacheKey != null) {
+ propStoreKey = PropStoreKey.fromPath(path);
+ if (propStoreKey != null) {
// notify listeners
- Set<PropChangeListener> snapshot = getListenerSnapshot(propCacheKey);
+ Set<PropChangeListener> snapshot = getListenerSnapshot(propStoreKey);
if (snapshot != null) {
executorService
- .execute(new PropStoreEventTask.PropStoreDeleteEventTask(propCacheKey, snapshot));
+ .execute(new PropStoreEventTask.PropStoreDeleteEventTask(propStoreKey, snapshot));
}
- listenerCleanup(propCacheKey);
+ listenerCleanup(propStoreKey);
}
break;
case None:
@@ -160,36 +160,36 @@ public class PropStoreWatcher implements Watcher {
}
/**
- * Execute a task to notify registered listeners that the propCacheKey node received an event
+ * Execute a task to notify registered listeners that the propStoreKey node received an event
* notification from ZooKeeper and should be updated. The process can be initiated either by a
* ZooKeeper notification or a change detected in the cache based on a ZooKeeper event.
*
- * @param propCacheKey
+ * @param propStoreKey
* the cache id
*/
- public void signalZkChangeEvent(@NonNull final PropCacheKey<?> propCacheKey) {
- log.trace("signal ZooKeeper change event: {}", propCacheKey);
- Set<PropChangeListener> snapshot = getListenerSnapshot(propCacheKey);
+ public void signalZkChangeEvent(@NonNull final PropStoreKey<?> propStoreKey) {
+ log.trace("signal ZooKeeper change event: {}", propStoreKey);
+ Set<PropChangeListener> snapshot = getListenerSnapshot(propStoreKey);
log.trace("Sending change event to: {}", snapshot);
if (snapshot != null) {
executorService
- .execute(new PropStoreEventTask.PropStoreZkChangeEventTask(propCacheKey, snapshot));
+ .execute(new PropStoreEventTask.PropStoreZkChangeEventTask(propStoreKey, snapshot));
}
}
/**
- * Execute a task to notify registered listeners that the propCacheKey node change was detected
+ * Execute a task to notify registered listeners that the propStoreKey node change was detected
* should be updated.
*
- * @param propCacheKey
+ * @param propStoreKey
* the cache id
*/
- public void signalCacheChangeEvent(final PropCacheKey<?> propCacheKey) {
- log.trace("cache change event: {}", propCacheKey);
- Set<PropChangeListener> snapshot = getListenerSnapshot(propCacheKey);
+ public void signalCacheChangeEvent(final PropStoreKey<?> propStoreKey) {
+ log.trace("cache change event: {}", propStoreKey);
+ Set<PropChangeListener> snapshot = getListenerSnapshot(propStoreKey);
if (snapshot != null) {
executorService
- .execute(new PropStoreEventTask.PropStoreCacheChangeEventTask(propCacheKey, snapshot));
+ .execute(new PropStoreEventTask.PropStoreCacheChangeEventTask(propStoreKey, snapshot));
}
}
@@ -197,13 +197,13 @@ public class PropStoreWatcher implements Watcher {
* Clean-up the active listeners set when an entry is removed from the cache, remove it from the
* active listeners.
*
- * @param propCacheKey
+ * @param propStoreKey
* the cache id
*/
- public void listenerCleanup(final PropCacheKey<?> propCacheKey) {
+ public void listenerCleanup(final PropStoreKey<?> propStoreKey) {
listenerWriteLock.lock();
try {
- listeners.remove(propCacheKey);
+ listeners.remove(propStoreKey);
} finally {
listenerWriteLock.unlock();
}
@@ -213,16 +213,16 @@ public class PropStoreWatcher implements Watcher {
* Get an immutable snapshot of the listeners for a prop cache id. The set is intended for
* notification of changes for a specific prop cache id.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache id
* @return an immutable copy of listeners.
*/
- private Set<PropChangeListener> getListenerSnapshot(final PropCacheKey<?> propCacheKey) {
+ private Set<PropChangeListener> getListenerSnapshot(final PropStoreKey<?> propStoreKey) {
Set<PropChangeListener> snapshot = null;
listenerReadLock.lock();
try {
- Set<PropChangeListener> set = listeners.get(propCacheKey);
+ Set<PropChangeListener> set = listeners.get(propStoreKey);
if (set != null) {
snapshot = Set.copyOf(set);
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java
index 803463e135..4761d66620 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoader.java
@@ -27,7 +27,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.checkerframework.checker.nullness.qual.Nullable;
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
import com.github.benmanes.caffeine.cache.CacheLoader;
-public class ZooPropLoader implements CacheLoader<PropCacheKey<?>,VersionedProperties> {
+public class ZooPropLoader implements CacheLoader<PropStoreKey<?>,VersionedProperties> {
private static final Logger log = LoggerFactory.getLogger(ZooPropLoader.class);
@@ -55,14 +55,14 @@ public class ZooPropLoader implements CacheLoader<PropCacheKey<?>,VersionedPrope
}
@Override
- public @Nullable VersionedProperties load(PropCacheKey<?> propCacheKey) {
+ public @Nullable VersionedProperties load(PropStoreKey<?> propStoreKey) {
try {
- log.trace("load called for {}", propCacheKey);
+ log.trace("load called for {}", propStoreKey);
long startNanos = System.nanoTime();
Stat stat = new Stat();
- byte[] bytes = zrw.getData(propCacheKey.getPath(), propStoreWatcher, stat);
+ byte[] bytes = zrw.getData(propStoreKey.getPath(), propStoreWatcher, stat);
VersionedProperties vProps = propCodec.fromBytes(stat.getVersion(), bytes);
metrics.addLoadTime(
@@ -71,41 +71,41 @@ public class ZooPropLoader implements CacheLoader<PropCacheKey<?>,VersionedPrope
return vProps;
} catch (KeeperException.NoNodeException ex) {
metrics.incrZkError();
- log.debug("property node for {} does not exist - it may be being created", propCacheKey);
- propStoreWatcher.signalZkChangeEvent(propCacheKey);
+ log.debug("property node for {} does not exist - it may be being created", propStoreKey);
+ propStoreWatcher.signalZkChangeEvent(propStoreKey);
return null;
} catch (Exception ex) {
metrics.incrZkError();
- log.info("Failed to load properties for: {} from ZooKeeper, returning null", propCacheKey,
+ log.info("Failed to load properties for: {} from ZooKeeper, returning null", propStoreKey,
ex);
- propStoreWatcher.signalZkChangeEvent(propCacheKey);
+ propStoreWatcher.signalZkChangeEvent(propStoreKey);
return null;
}
}
@Override
- public CompletableFuture<? extends VersionedProperties> asyncLoad(PropCacheKey<?> propCacheKey,
+ public CompletableFuture<? extends VersionedProperties> asyncLoad(PropStoreKey<?> propStoreKey,
Executor executor) throws Exception {
- log.trace("asyncLoad called for key: {}", propCacheKey);
- return CacheLoader.super.asyncLoad(propCacheKey, executor);
+ log.trace("asyncLoad called for key: {}", propStoreKey);
+ return CacheLoader.super.asyncLoad(propStoreKey, executor);
}
@Override
- public CompletableFuture<VersionedProperties> asyncReload(PropCacheKey<?> propCacheKey,
+ public CompletableFuture<VersionedProperties> asyncReload(PropStoreKey<?> propStoreKey,
VersionedProperties oldValue, Executor executor) throws Exception {
- log.trace("asyncReload called for key: {}", propCacheKey);
+ log.trace("asyncReload called for key: {}", propStoreKey);
metrics.incrRefresh();
- return CompletableFuture.supplyAsync(() -> loadIfDifferentVersion(propCacheKey, oldValue),
+ return CompletableFuture.supplyAsync(() -> loadIfDifferentVersion(propStoreKey, oldValue),
executor);
}
@Override
- public @Nullable VersionedProperties reload(PropCacheKey<?> propCacheKey,
+ public @Nullable VersionedProperties reload(PropStoreKey<?> propStoreKey,
VersionedProperties oldValue) throws Exception {
- log.trace("reload called for: {}", propCacheKey);
+ log.trace("reload called for: {}", propStoreKey);
metrics.incrRefresh();
- return loadIfDifferentVersion(propCacheKey, oldValue);
+ return loadIfDifferentVersion(propStoreKey, oldValue);
}
/**
@@ -122,7 +122,7 @@ public class ZooPropLoader implements CacheLoader<PropCacheKey<?>,VersionedPrope
* @return versioned properties that match the values stored in ZooKeeper, or null if the
* properties cannot be retrieved.
*/
- private @Nullable VersionedProperties loadIfDifferentVersion(PropCacheKey<?> propCacheId,
+ private @Nullable VersionedProperties loadIfDifferentVersion(PropStoreKey<?> propCacheId,
VersionedProperties currentValue) {
requireNonNull(propCacheId, "propCacheId cannot be null");
try {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
index c523574de3..073a7dfc1e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
@@ -34,9 +34,9 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
import org.apache.accumulo.server.conf.store.PropCache;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
import org.apache.accumulo.server.conf.store.PropChangeListener;
import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.accumulo.server.conf.store.SystemPropKey;
import org.apache.accumulo.server.conf.util.ConfigTransformer;
import org.apache.zookeeper.KeeperException;
@@ -55,7 +55,7 @@ public class ZooPropStore implements PropStore, PropChangeListener {
private final ZooReaderWriter zrw;
private final PropStoreWatcher propStoreWatcher;
- private final PropCache cache;
+ private final PropCacheCaffeineImpl cache;
private final PropStoreMetrics cacheMetrics = new PropStoreMetrics();
private final ReadyMonitor zkReadyMon;
@@ -129,9 +129,9 @@ public class ZooPropStore implements PropStore, PropChangeListener {
}
@Override
- public boolean exists(final PropCacheKey<?> propCacheKey) {
+ public boolean exists(final PropStoreKey<?> propStoreKey) {
try {
- if (zrw.exists(propCacheKey.getPath())) {
+ if (zrw.exists(propStoreKey.getPath())) {
return true;
}
} catch (KeeperException ex) {
@@ -148,14 +148,14 @@ public class ZooPropStore implements PropStore, PropChangeListener {
}
@Override
- public void create(PropCacheKey<?> propCacheKey, Map<String,String> props) {
+ public void create(PropStoreKey<?> propStoreKey, Map<String,String> props) {
try {
VersionedProperties vProps = new VersionedProperties(props);
- String path = propCacheKey.getPath();
+ String path = propStoreKey.getPath();
zrw.putPrivatePersistentData(path, codec.toBytes(vProps), ZooUtil.NodeExistsPolicy.FAIL);
} catch (IOException | KeeperException | InterruptedException ex) {
- throw new IllegalStateException("Failed to serialize properties for " + propCacheKey, ex);
+ throw new IllegalStateException("Failed to serialize properties for " + propStoreKey, ex);
}
}
@@ -164,28 +164,28 @@ public class ZooPropStore implements PropStore, PropChangeListener {
* legacy properties exist, they will be converted to the new storage form and naming convention.
* The legacy properties are deleted once the new node format is written.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @return The versioned properties.
* @throws IllegalStateException
* if the updates fails because of an underlying store exception
*/
@Override
- public @NonNull VersionedProperties get(final PropCacheKey<?> propCacheKey) {
+ public @NonNull VersionedProperties get(final PropStoreKey<?> propStoreKey) {
checkZkConnection(); // if ZK not connected, block, do not just return a cached value.
- propStoreWatcher.registerListener(propCacheKey, this);
+ propStoreWatcher.registerListener(propStoreKey, this);
- var props = cache.get(propCacheKey);
+ var props = cache.get(propStoreKey);
if (props != null) {
return props;
}
- if (propCacheKey instanceof SystemPropKey) {
- return new ConfigTransformer(zrw, codec, propStoreWatcher).transform(propCacheKey);
+ if (propStoreKey instanceof SystemPropKey) {
+ return new ConfigTransformer(zrw, codec, propStoreWatcher).transform(propStoreKey);
}
throw new IllegalStateException(
- "Invalid request for " + propCacheKey + ", the property node does not exist");
+ "Invalid request for " + propStoreKey + ", the property node does not exist");
}
/**
@@ -193,7 +193,7 @@ public class ZooPropStore implements PropStore, PropChangeListener {
* directly from ZooKeeper. This allows utilities access when there is a ZooKeeper, may there may
* not be a full instance running. All exception handling is left to the caller.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache key
* @param watcher
* a prop store watcher that will receive / handle ZooKeeper events.
@@ -207,12 +207,12 @@ public class ZooPropStore implements PropStore, PropChangeListener {
* @throws InterruptedException
* if the ZooKeeper read was interrupted.
*/
- public static @Nullable VersionedProperties readFromZk(final PropCacheKey<?> propCacheKey,
+ public static @Nullable VersionedProperties readFromZk(final PropStoreKey<?> propStoreKey,
final PropStoreWatcher watcher, final ZooReader zooReader)
throws IOException, KeeperException, InterruptedException {
try {
Stat stat = new Stat();
- byte[] bytes = zooReader.getData(propCacheKey.getPath(), watcher, stat);
+ byte[] bytes = zooReader.getData(propStoreKey.getPath(), watcher, stat);
return codec.fromBytes(stat.getVersion(), bytes);
} catch (KeeperException.NoNodeException ex) {
// ignore no node - allow other exceptions to propagate
@@ -231,7 +231,7 @@ public class ZooPropStore implements PropStore, PropChangeListener {
* value(s) will be the set to the values provided by the last thread to complete. The order is
* indeterminate.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache id
* @param props
* a map of property k,v pairs
@@ -239,77 +239,77 @@ public class ZooPropStore implements PropStore, PropChangeListener {
* if the values cannot be written or if an underlying store exception occurs.
*/
@Override
- public void putAll(@NonNull PropCacheKey<?> propCacheKey, @NonNull Map<String,String> props) {
+ public void putAll(@NonNull PropStoreKey<?> propStoreKey, @NonNull Map<String,String> props) {
if (props.isEmpty()) {
return; // no props - noop
}
- mutateVersionedProps(propCacheKey, VersionedProperties::addOrUpdate, props);
+ mutateVersionedProps(propStoreKey, VersionedProperties::addOrUpdate, props);
}
@Override
- public void removeProperties(@NonNull PropCacheKey<?> propCacheKey,
+ public void removeProperties(@NonNull PropStoreKey<?> propStoreKey,
@NonNull Collection<String> keys) {
if (keys.isEmpty()) {
return; // no keys - noop.
}
- mutateVersionedProps(propCacheKey, VersionedProperties::remove, keys);
+ mutateVersionedProps(propStoreKey, VersionedProperties::remove, keys);
}
@Override
- public void delete(@NonNull PropCacheKey<?> propCacheKey) {
- Objects.requireNonNull(propCacheKey, "prop store delete() - Must provide propCacheId");
+ public void delete(@NonNull PropStoreKey<?> propStoreKey) {
+ Objects.requireNonNull(propStoreKey, "prop store delete() - Must provide propCacheId");
try {
- log.trace("called delete() for: {}", propCacheKey);
- final String path = propCacheKey.getPath();
+ log.trace("called delete() for: {}", propStoreKey);
+ final String path = propStoreKey.getPath();
zrw.delete(path);
- cache.remove(propCacheKey);
+ cache.remove(propStoreKey);
} catch (KeeperException | InterruptedException ex) {
- throw new IllegalStateException("Failed to delete properties for propCacheId " + propCacheKey,
+ throw new IllegalStateException("Failed to delete properties for propCacheId " + propStoreKey,
ex);
}
}
- private <T> void mutateVersionedProps(PropCacheKey<?> propCacheKey,
+ private <T> void mutateVersionedProps(PropStoreKey<?> propStoreKey,
BiFunction<VersionedProperties,T,VersionedProperties> action, T changes) {
- log.trace("mutateVersionedProps called for: {}", propCacheKey);
+ log.trace("mutateVersionedProps called for: {}", propStoreKey);
try {
- VersionedProperties vProps = cache.getWithoutCaching(propCacheKey);
+ VersionedProperties vProps = cache.getWithoutCaching(propStoreKey);
if (vProps == null) {
- vProps = readPropsFromZk(propCacheKey);
+ vProps = readPropsFromZk(propStoreKey);
}
for (int attempts = 3; attempts > 0; --attempts) {
VersionedProperties updates = action.apply(vProps, changes);
- if (zrw.overwritePersistentData(propCacheKey.getPath(), codec.toBytes(updates),
+ if (zrw.overwritePersistentData(propStoreKey.getPath(), codec.toBytes(updates),
(int) updates.getDataVersion())) {
return;
}
Thread.sleep(20); // small pause to get thread to yield.
// re-read from zookeeper to ensure the latest version.
- vProps = readPropsFromZk(propCacheKey);
+ vProps = readPropsFromZk(propStoreKey);
}
throw new IllegalStateException(
- "failed to remove properties to zooKeeper for " + propCacheKey, null);
+ "failed to remove properties to zooKeeper for " + propStoreKey, null);
} catch (IllegalArgumentException | IOException ex) {
throw new IllegalStateException(
- "Codec failed to decode / encode properties for " + propCacheKey, ex);
+ "Codec failed to decode / encode properties for " + propStoreKey, ex);
} catch (InterruptedException | KeeperException ex) {
if (ex instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw new IllegalStateException(
- "failed to remove properties to zooKeeper for " + propCacheKey, ex);
+ "failed to remove properties to zooKeeper for " + propStoreKey, ex);
}
}
@Override
- public void registerAsListener(PropCacheKey<?> propCacheKey, PropChangeListener listener) {
- propStoreWatcher.registerListener(propCacheKey, listener);
+ public void registerAsListener(PropStoreKey<?> propStoreKey, PropChangeListener listener) {
+ propStoreWatcher.registerListener(propStoreKey, listener);
}
private void checkZkConnection() {
@@ -325,9 +325,9 @@ public class ZooPropStore implements PropStore, PropChangeListener {
}
@Override
- public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
- log.trace("Received change event from ZooKeeper for: {} removed from cache", propCacheKey);
- cache.remove(propCacheKey);
+ public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
+ log.trace("Received change event from ZooKeeper for: {} removed from cache", propStoreKey);
+ cache.remove(propStoreKey);
}
/**
@@ -335,18 +335,18 @@ public class ZooPropStore implements PropStore, PropChangeListener {
* change is also sent to external listeners of the need to take action, but for the prop store,
* no additional action is required.
*
- * @param propCacheKey
+ * @param propStoreKey
* the prop cache id.
*/
@Override
- public void cacheChangeEvent(PropCacheKey<?> propCacheKey) {
- log.trace("zkChangeEvent: {}", propCacheKey);
+ public void cacheChangeEvent(PropStoreKey<?> propStoreKey) {
+ log.trace("zkChangeEvent: {}", propStoreKey);
}
@Override
- public void deleteEvent(PropCacheKey<?> propCacheKey) {
- log.trace("deleteEvent: {}", propCacheKey);
- cache.remove(propCacheKey);
+ public void deleteEvent(PropStoreKey<?> propStoreKey) {
+ log.trace("deleteEvent: {}", propStoreKey);
+ cache.remove(propStoreKey);
}
@Override
@@ -358,8 +358,8 @@ public class ZooPropStore implements PropStore, PropChangeListener {
/**
* Read and decode property node from ZooKeeper.
*
- * @param propCacheKey
- * the propCacheKey
+ * @param propStoreKey
+ * the propStoreKey
* @return the decoded properties.
* @throws KeeperException
* if a ZooKeeper exception occurs to allow caller to decide on action.
@@ -370,15 +370,25 @@ public class ZooPropStore implements PropStore, PropChangeListener {
* if an interrupt occurs. The interrupt status is reasserted and usually best to not
* otherwise try to handle the exception.
*/
- private VersionedProperties readPropsFromZk(PropCacheKey<?> propCacheKey)
+ private VersionedProperties readPropsFromZk(PropStoreKey<?> propStoreKey)
throws KeeperException, IOException {
try {
Stat stat = new Stat();
- byte[] bytes = zrw.getData(propCacheKey.getPath(), stat);
+ byte[] bytes = zrw.getData(propStoreKey.getPath(), stat);
return codec.fromBytes(stat.getVersion(), bytes);
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
throw new IllegalStateException("Interrupt received during ZooKeeper read", ex);
}
}
+
+ @Override
+ public PropCache getCache() {
+ return cache;
+ }
+
+ @Override
+ public @Nullable VersionedProperties getWithoutCaching(PropStoreKey<?> propStoreKey) {
+ return cache.getWithoutCaching(propStoreKey);
+ }
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java
index 73fa29324d..8912250eb5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java
@@ -40,7 +40,7 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.fate.zookeeper.ZooUtil;
import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.accumulo.server.conf.store.SystemPropKey;
import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
@@ -104,27 +104,27 @@ public class ConfigTransformer {
*
* @return the encoded properties.
*/
- public VersionedProperties transform(final PropCacheKey<?> propCacheKey) {
- TransformToken token = TransformToken.createToken(propCacheKey, zrw);
- return transform(propCacheKey, token);
+ public VersionedProperties transform(final PropStoreKey<?> propStoreKey) {
+ TransformToken token = TransformToken.createToken(propStoreKey, zrw);
+ return transform(propStoreKey, token);
}
// Allow external (mocked) TransformToken to be used
@VisibleForTesting
- VersionedProperties transform(final PropCacheKey<?> propCacheKey, final TransformToken token) {
+ VersionedProperties transform(final PropStoreKey<?> propStoreKey, final TransformToken token) {
- log.info("checking for legacy property upgrade transform for {}", propCacheKey);
+ log.info("checking for legacy property upgrade transform for {}", propStoreKey);
VersionedProperties results;
Instant start = Instant.now();
try {
// check for node - just return if it exists.
- results = ZooPropStore.readFromZk(propCacheKey, propStoreWatcher, zrw);
+ results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
if (results != null) {
log.debug(
"Found existing node at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
- propCacheKey, results.getDataVersion(), results.getTimestamp());
+ propStoreKey, results.getDataVersion(), results.getTimestamp());
return results;
}
@@ -134,67 +134,67 @@ public class ConfigTransformer {
retry.waitForNextAttempt();
// look and return node if created while trying to token.
log.trace("own the token - look for existing encoded node at: {}",
- propCacheKey.getPath());
- results = ZooPropStore.readFromZk(propCacheKey, propStoreWatcher, zrw);
+ propStoreKey.getPath());
+ results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
if (results != null) {
log.debug(
"Found existing node after getting token at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
- propCacheKey, results.getDataVersion(), results.getTimestamp());
+ propStoreKey, results.getDataVersion(), results.getTimestamp());
return results;
}
// still does not exist - try again.
token.getTokenOwnership();
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
- throw new IllegalStateException("Failed to hold transform token for " + propCacheKey, ex);
+ throw new IllegalStateException("Failed to hold transform token for " + propStoreKey, ex);
} catch (IllegalStateException ex) {
- throw new IllegalStateException("Failed to hold transform token for " + propCacheKey, ex);
+ throw new IllegalStateException("Failed to hold transform token for " + propStoreKey, ex);
}
}
- Set<LegacyPropNode> upgradeNodes = readLegacyProps(propCacheKey);
+ Set<LegacyPropNode> upgradeNodes = readLegacyProps(propStoreKey);
if (upgradeNodes == null) {
log.info("Found existing node after reading legacy props {}, skipping conversion",
- propCacheKey);
- results = ZooPropStore.readFromZk(propCacheKey, propStoreWatcher, zrw);
+ propStoreKey);
+ results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
if (results != null) {
return results;
}
}
- upgradeNodes = convertDeprecatedProps(propCacheKey, upgradeNodes);
+ upgradeNodes = convertDeprecatedProps(propStoreKey, upgradeNodes);
- results = writeConverted(propCacheKey, upgradeNodes);
+ results = writeConverted(propStoreKey, upgradeNodes);
if (results == null) {
- throw new IllegalStateException("Could not create properties for " + propCacheKey);
+ throw new IllegalStateException("Could not create properties for " + propStoreKey);
}
// validate token still valid before deletion.
if (!token.validateToken()) {
throw new IllegalStateException(
- "legacy conversion failed. Lost transform token for " + propCacheKey);
+ "legacy conversion failed. Lost transform token for " + propStoreKey);
}
int errorCount = deleteLegacyProps(upgradeNodes);
log.debug("deleted legacy props - error count: {}", errorCount);
- log.debug("property transform for {} took {} ms", propCacheKey,
+ log.debug("property transform for {} took {} ms", propStoreKey,
new DurationFormat(Duration.between(start, Instant.now()).toMillis(), ""));
return results;
} catch (Exception ex) {
- log.info("Exception on upgrading legacy properties for: " + propCacheKey, ex);
+ log.info("Exception on upgrading legacy properties for: " + propStoreKey, ex);
} finally {
token.releaseToken();
}
return null;
}
- private Set<LegacyPropNode> convertDeprecatedProps(PropCacheKey<?> propCacheKey,
+ private Set<LegacyPropNode> convertDeprecatedProps(PropStoreKey<?> propStoreKey,
Set<LegacyPropNode> upgradeNodes) {
- if (!(propCacheKey instanceof SystemPropKey)) {
+ if (!(propStoreKey instanceof SystemPropKey)) {
return upgradeNodes;
}
@@ -212,7 +212,7 @@ public class ConfigTransformer {
return renamedNodes;
}
- private @Nullable Set<LegacyPropNode> readLegacyProps(PropCacheKey<?> propCacheKey) {
+ private @Nullable Set<LegacyPropNode> readLegacyProps(PropStoreKey<?> propStoreKey) {
Set<LegacyPropNode> legacyProps = new TreeSet<>();
@@ -220,17 +220,17 @@ public class ConfigTransformer {
var tokenName = TransformToken.TRANSFORM_TOKEN.substring(1);
try {
- var keyBasePath = propCacheKey.getBasePath();
+ var keyBasePath = propStoreKey.getBasePath();
List<String> childNames = zrw.getChildren(keyBasePath);
for (String propName : childNames) {
- log.trace("processing ZooKeeper child node: {} for: {}", propName, propCacheKey);
+ log.trace("processing ZooKeeper child node: {} for: {}", propName, propStoreKey);
if (tokenName.equals(propName)) {
continue;
}
- if (PropCacheKey.PROP_NODE_NAME.equals(propName)) {
+ if (PropStoreKey.PROP_NODE_NAME.equals(propName)) {
log.debug(
"encoded property node exists for {}. Legacy conversion ignoring conversion of this node",
- propCacheKey);
+ propStoreKey);
return null;
}
log.trace("Adding: {} to list for legacy conversion", propName);
@@ -278,26 +278,26 @@ public class ConfigTransformer {
return errorCount;
}
- private @Nullable VersionedProperties writeConverted(final PropCacheKey<?> propCacheKey,
+ private @Nullable VersionedProperties writeConverted(final PropStoreKey<?> propStoreKey,
final Set<LegacyPropNode> nodes) {
final Map<String,String> props = new HashMap<>();
nodes.forEach(node -> props.put(node.getPropName(), node.getData()));
VersionedProperties vProps = new VersionedProperties(props);
- String path = propCacheKey.getPath();
+ String path = propStoreKey.getPath();
try {
try {
zrw.putPrivatePersistentData(path, codec.toBytes(vProps), ZooUtil.NodeExistsPolicy.FAIL);
} catch (KeeperException.NodeExistsException ex) {
- vProps = ZooPropStore.readFromZk(propCacheKey, propStoreWatcher, zrw);
+ vProps = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
}
} catch (InterruptedException | IOException | KeeperException ex) {
if (ex instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw new IllegalStateException(
- "failed to create node for " + propCacheKey + " on conversion", ex);
+ "failed to create node for " + propStoreKey + " on conversion", ex);
}
- if (!validateWrite(propCacheKey, vProps)) {
+ if (!validateWrite(propStoreKey, vProps)) {
// failed validation
return null;
}
@@ -305,20 +305,20 @@ public class ConfigTransformer {
return vProps;
}
- private boolean validateWrite(final PropCacheKey<?> propCacheKey,
+ private boolean validateWrite(final PropStoreKey<?> propStoreKey,
final VersionedProperties vProps) {
try {
- Stat stat = zrw.getStatus(propCacheKey.getPath(), propStoreWatcher);
+ Stat stat = zrw.getStatus(propStoreKey.getPath(), propStoreWatcher);
if (stat == null) {
throw new IllegalStateException(
- "failed to get stat to validate created node for " + propCacheKey);
+ "failed to get stat to validate created node for " + propStoreKey);
}
return stat.getVersion() == vProps.getDataVersion();
} catch (KeeperException ex) {
- throw new IllegalStateException("failed to validate created node for " + propCacheKey, ex);
+ throw new IllegalStateException("failed to validate created node for " + propStoreKey, ex);
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
- throw new IllegalStateException("failed to validate created node for " + propCacheKey, ex);
+ throw new IllegalStateException("failed to validate created node for " + propStoreKey, ex);
}
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/PropSnapshot.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/PropSnapshot.java
index 28cc7371c0..f1aa2f6dd9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/PropSnapshot.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/PropSnapshot.java
@@ -24,9 +24,9 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
import org.apache.accumulo.server.conf.store.PropChangeListener;
import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -46,17 +46,17 @@ public class PropSnapshot implements PropChangeListener {
private final Lock updateLock = new ReentrantLock();
private final AtomicBoolean needsUpdate = new AtomicBoolean(true);
private final AtomicReference<VersionedProperties> vPropRef = new AtomicReference<>();
- private final PropCacheKey<?> propCacheKey;
+ private final PropStoreKey<?> propStoreKey;
private final PropStore propStore;
- public static PropSnapshot create(final PropCacheKey<?> propCacheKey, final PropStore propStore) {
- var ps = new PropSnapshot(propCacheKey, propStore);
- propStore.registerAsListener(propCacheKey, ps);
+ public static PropSnapshot create(final PropStoreKey<?> propStoreKey, final PropStore propStore) {
+ var ps = new PropSnapshot(propStoreKey, propStore);
+ propStore.registerAsListener(propStoreKey, ps);
return ps;
}
- private PropSnapshot(final PropCacheKey<?> propCacheKey, final PropStore propStore) {
- this.propCacheKey = propCacheKey;
+ private PropSnapshot(final PropStoreKey<?> propStoreKey, final PropStore propStore) {
+ this.propStoreKey = propStoreKey;
this.propStore = propStore;
}
@@ -102,7 +102,7 @@ public class PropSnapshot implements PropChangeListener {
if (!needsUpdate.get()) {
return;
}
- var vProps = propStore.get(propCacheKey);
+ var vProps = propStore.get(propStoreKey);
vPropRef.set(vProps);
needsUpdate.set(false);
} finally {
@@ -111,24 +111,24 @@ public class PropSnapshot implements PropChangeListener {
}
@Override
- public void zkChangeEvent(final PropCacheKey<?> eventPropKey) {
- if (propCacheKey.equals(eventPropKey)) {
+ public void zkChangeEvent(final PropStoreKey<?> eventPropKey) {
+ if (propStoreKey.equals(eventPropKey)) {
requireUpdate();
}
}
@Override
- public void cacheChangeEvent(final PropCacheKey<?> eventPropKey) {
- if (propCacheKey.equals(eventPropKey)) {
+ public void cacheChangeEvent(final PropStoreKey<?> eventPropKey) {
+ if (propStoreKey.equals(eventPropKey)) {
requireUpdate();
}
}
@Override
- public void deleteEvent(final PropCacheKey<?> eventPropKey) {
- if (propCacheKey.equals(eventPropKey)) {
+ public void deleteEvent(final PropStoreKey<?> eventPropKey) {
+ if (propStoreKey.equals(eventPropKey)) {
requireUpdate();
- log.debug("Received property delete event for {}", propCacheKey);
+ log.debug("Received property delete event for {}", propStoreKey);
}
}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java
index 943fc8fa7a..c7eb855530 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java
@@ -25,7 +25,7 @@ import java.util.Objects;
import java.util.UUID;
import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.checkerframework.checker.nullness.qual.NonNull;
@@ -55,7 +55,7 @@ public class TransformToken {
private final ZooReaderWriter zrw;
private boolean haveToken = false;
- private TransformToken(final @NonNull PropCacheKey<?> key, final ZooReaderWriter zrw) {
+ private TransformToken(final @NonNull PropStoreKey<?> key, final ZooReaderWriter zrw) {
path = key.getBasePath() + TRANSFORM_TOKEN;
this.zrw = zrw;
@@ -70,7 +70,7 @@ public class TransformToken {
* to be created by the thread that created the lock, or try calling to {@code lock} to succeed
*
* @param key
- * a PropCacheKey that defines the storage location of the created lock and the
+ * a PropStoreKey that defines the storage location of the created lock and the
* associated property nodes.
* @param zrw
* a ZooReaderWriter
@@ -78,7 +78,7 @@ public class TransformToken {
* @throws IllegalStateException
* is the lock creation fails due to an underlying ZooKeeper exception.
*/
- public static TransformToken createToken(final @NonNull PropCacheKey<?> key,
+ public static TransformToken createToken(final @NonNull PropStoreKey<?> key,
final ZooReaderWriter zrw) {
return new TransformToken(key, zrw);
}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
index c57d4248de..621e10f42f 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
@@ -75,10 +75,10 @@ public class NamespaceConfigurationTest {
parent = createMock(AccumuloConfiguration.class);
reset(propStore);
- var nsCacheKey = NamespacePropKey.of(iid, NSID);
- expect(propStore.get(eq(nsCacheKey))).andReturn(new VersionedProperties(123, Instant.now(),
+ var nsPropStoreKey = NamespacePropKey.of(iid, NSID);
+ expect(propStore.get(eq(nsPropStoreKey))).andReturn(new VersionedProperties(123, Instant.now(),
Map.of(Property.INSTANCE_SECRET.getKey(), "sekrit"))).anyTimes();
- propStore.registerAsListener(eq(nsCacheKey), anyObject());
+ propStore.registerAsListener(eq(nsPropStoreKey), anyObject());
expectLastCall().anyTimes();
replay(propStore, context);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropCacheKeyTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
similarity index 92%
rename from server/base/src/test/java/org/apache/accumulo/server/conf/store/PropCacheKeyTest.java
rename to server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
index 39982dc5fb..15de7736c2 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropCacheKeyTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/PropStoreKeyTest.java
@@ -21,7 +21,7 @@ package org.apache.accumulo.server.conf.store;
import static org.apache.accumulo.core.Constants.ZCONFIG;
import static org.apache.accumulo.core.Constants.ZNAMESPACES;
import static org.apache.accumulo.core.Constants.ZTABLES;
-import static org.apache.accumulo.server.conf.store.PropCacheKey.PROP_NODE_NAME;
+import static org.apache.accumulo.server.conf.store.PropStoreKey.PROP_NODE_NAME;
import static org.easymock.EasyMock.createMock;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.replay;
@@ -40,8 +40,8 @@ import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class PropCacheKeyTest {
- private static final Logger log = LoggerFactory.getLogger(PropCacheKeyTest.class);
+public class PropStoreKeyTest {
+ private static final Logger log = LoggerFactory.getLogger(PropStoreKeyTest.class);
private final InstanceId instanceId = InstanceId.of(UUID.randomUUID());
@@ -98,18 +98,18 @@ public class PropCacheKeyTest {
var iid = "3f9976c6-3bf1-41ab-9751-1b0a9be3551d";
- PropCacheKey<?> t1 =
- PropCacheKey.fromPath("/accumulo/" + iid + "/tables/t1/conf/encoded_props");
+ PropStoreKey<?> t1 =
+ PropStoreKey.fromPath("/accumulo/" + iid + "/tables/t1/conf/encoded_props");
assertNotNull(t1);
assertEquals(TableId.of("t1"), t1.getId());
- PropCacheKey<?> n1 =
- PropCacheKey.fromPath("/accumulo/" + iid + "/namespaces/n1/conf/encoded_props");
+ PropStoreKey<?> n1 =
+ PropStoreKey.fromPath("/accumulo/" + iid + "/namespaces/n1/conf/encoded_props");
assertNotNull(n1);
assertEquals(NamespaceId.of("n1"), n1.getId());
assertNotNull(n1.getId());
- PropCacheKey<?> s1 = PropCacheKey.fromPath("/accumulo/" + iid + "/config/encoded_props");
+ PropStoreKey<?> s1 = PropStoreKey.fromPath("/accumulo/" + iid + "/config/encoded_props");
assertNotNull(s1);
// system config returns instance id as id placeholder
assertEquals(iid, s1.getId().canonical());
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java
index 59d28c3c77..02ad89061c 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/PropStoreEventTest.java
@@ -43,9 +43,9 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.server.ServerContext;
import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
import org.apache.accumulo.server.conf.store.PropChangeListener;
import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.accumulo.server.conf.store.TablePropKey;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
@@ -280,17 +280,17 @@ public class PropStoreEventTest {
private int connectionEventCount = 0;
@Override
- public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
+ public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
zkChangeEventCount++;
}
@Override
- public void cacheChangeEvent(PropCacheKey<?> propCacheKey) {
+ public void cacheChangeEvent(PropStoreKey<?> propStoreKey) {
cacheChangeEventCount++;
}
@Override
- public void deleteEvent(PropCacheKey<?> propCacheKey) {
+ public void deleteEvent(PropStoreKey<?> propStoreKey) {
deleteEventCount++;
}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java
index ba376dc22d..2487513c13 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropLoaderTest.java
@@ -51,7 +51,7 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.server.ServerContext;
import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.accumulo.server.conf.store.SystemPropKey;
import org.apache.accumulo.server.conf.store.TablePropKey;
import org.apache.zookeeper.KeeperException;
@@ -70,7 +70,7 @@ public class ZooPropLoaderTest {
private PropCacheCaffeineImplTest.TestTicker ticker;
private InstanceId instanceId;
private ServerContext context;
- private PropCacheKey<?> propCacheKey;
+ private PropStoreKey<?> propStoreKey;
private VersionedPropCodec propCodec;
// mocks
@@ -85,7 +85,7 @@ public class ZooPropLoaderTest {
ticker = new PropCacheCaffeineImplTest.TestTicker();
instanceId = InstanceId.of(UUID.randomUUID());
- propCacheKey = TablePropKey.of(instanceId, TableId.of("1abc"));
+ propStoreKey = TablePropKey.of(instanceId, TableId.of("1abc"));
propCodec = VersionedPropCodec.getDefault();
// mocks
@@ -113,7 +113,7 @@ public class ZooPropLoaderTest {
VersionedProperties defaultProps = new VersionedProperties();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
.andReturn(propCodec.toBytes(defaultProps)).anyTimes();
cacheMetrics.addLoadTime(anyLong());
@@ -121,7 +121,7 @@ public class ZooPropLoaderTest {
replay(context, zrw, propStoreWatcher, cacheMetrics);
- assertNotNull(loader.load(propCacheKey));
+ assertNotNull(loader.load(propStoreKey));
}
// from cache loader
@@ -137,9 +137,9 @@ public class ZooPropLoaderTest {
VersionedProperties defaultProps = new VersionedProperties();
- expect(zrw.getStatus(propCacheKey.getPath())).andThrow(new KeeperException.NoNodeException())
+ expect(zrw.getStatus(propStoreKey.getPath())).andThrow(new KeeperException.NoNodeException())
.anyTimes();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
.andReturn(propCodec.toBytes(defaultProps)).once();
cacheMetrics.addLoadTime(anyLong());
@@ -151,11 +151,11 @@ public class ZooPropLoaderTest {
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
// load into cache
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
// read cached entry - load count should not change.
ticker.advance(1, TimeUnit.MINUTES);
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
}
@Test
@@ -174,10 +174,10 @@ public class ZooPropLoaderTest {
@Test
public void loadFailTest() throws Exception {
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
.andThrow(new KeeperException.NoNodeException("force no node exception")).once();
- propStoreWatcher.signalZkChangeEvent(eq(propCacheKey));
+ propStoreWatcher.signalZkChangeEvent(eq(propStoreKey));
expectLastCall();
cacheMetrics.incrZkError();
@@ -188,7 +188,7 @@ public class ZooPropLoaderTest {
PropCacheCaffeineImpl cache =
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
- assertNull(cache.get(propCacheKey));
+ assertNull(cache.get(propStoreKey));
log.debug("Metrics: {}", cacheMetrics);
}
@@ -203,7 +203,7 @@ public class ZooPropLoaderTest {
VersionedProperties defaultProps = new VersionedProperties();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
.andReturn(propCodec.toBytes(defaultProps)).times(2);
cacheMetrics.addLoadTime(anyLong());
@@ -218,11 +218,11 @@ public class ZooPropLoaderTest {
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
// load cache
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
ticker.advance(70, TimeUnit.MINUTES);
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
}
@@ -238,14 +238,14 @@ public class ZooPropLoaderTest {
final VersionedProperties defaultProps = new VersionedProperties();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
.andReturn(propCodec.toBytes(defaultProps)).once();
Stat stat = new Stat();
stat.setVersion(123); // set different version so reload triggered
- expect(zrw.getStatus(propCacheKey.getPath())).andReturn(stat).once();
+ expect(zrw.getStatus(propStoreKey.getPath())).andReturn(stat).once();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
.andThrow(new KeeperException.NoNodeException("forced no node")).anyTimes();
propStoreWatcher.signalZkChangeEvent(anyObject());
@@ -269,19 +269,19 @@ public class ZooPropLoaderTest {
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
// prime cache
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
ticker.advance(5, TimeUnit.MINUTES);
// read cached value
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
// advance so refresh called.
ticker.advance(20, TimeUnit.MINUTES);
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
- assertNull(cache.get(propCacheKey));
+ assertNull(cache.get(propStoreKey));
}
@Test
@@ -292,7 +292,7 @@ public class ZooPropLoaderTest {
PropCacheCaffeineImpl cache =
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
- assertNull(cache.getWithoutCaching(propCacheKey));
+ assertNull(cache.getWithoutCaching(propStoreKey));
}
@@ -366,7 +366,7 @@ public class ZooPropLoaderTest {
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
// load into cache
- assertNull(cache.getWithoutCaching(propCacheKey));
+ assertNull(cache.getWithoutCaching(propStoreKey));
}
@Test
@@ -376,7 +376,7 @@ public class ZooPropLoaderTest {
// first call loads cache
Capture<Stat> stat = newCapture();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
Stat s = stat.getValue();
s.setCtime(System.currentTimeMillis());
s.setMtime(System.currentTimeMillis());
@@ -388,7 +388,7 @@ public class ZooPropLoaderTest {
Stat expectedStat = new Stat();
expectedStat.setVersion(0);
- expect(zrw.getStatus(propCacheKey.getPath())).andReturn(expectedStat).times(2);
+ expect(zrw.getStatus(propStoreKey.getPath())).andReturn(expectedStat).times(2);
cacheMetrics.addLoadTime(anyLong());
expectLastCall().times(1);
@@ -401,23 +401,23 @@ public class ZooPropLoaderTest {
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
// load cache
- log.debug("received: {}", cache.get(propCacheKey));
+ log.debug("received: {}", cache.get(propStoreKey));
ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
ticker.advance(REFRESH_MIN / 2, TimeUnit.MINUTES);
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
ticker.advance(1, TimeUnit.MINUTES);
- assertNotNull(cache.get(propCacheKey));
+ assertNotNull(cache.get(propStoreKey));
}
@@ -433,7 +433,7 @@ public class ZooPropLoaderTest {
Capture<Stat> stat = newCapture();
- expect(zrw.getData(eq(propCacheKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
+ expect(zrw.getData(eq(propStoreKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
.andAnswer(() -> {
Stat s = stat.getValue();
s.setCtime(System.currentTimeMillis());
@@ -447,12 +447,12 @@ public class ZooPropLoaderTest {
// make it look like version on ZK has advanced.
Stat stat2 = new Stat();
stat2.setVersion(initialVersion + 3); // initSysProps 123, on write 124
- expect(zrw.getStatus(propCacheKey.getPath())).andReturn(stat2).once();
+ expect(zrw.getStatus(propStoreKey.getPath())).andReturn(stat2).once();
Capture<Stat> stat3 = newCapture();
expect(
- zrw.getData(eq(propCacheKey.getPath()), capture(propStoreWatcherCapture), capture(stat3)))
+ zrw.getData(eq(propStoreKey.getPath()), capture(propStoreWatcherCapture), capture(stat3)))
.andAnswer(() -> {
Stat s = stat3.getValue();
s.setCtime(System.currentTimeMillis());
@@ -463,7 +463,7 @@ public class ZooPropLoaderTest {
Map.of(Property.TABLE_SPLIT_THRESHOLD.getKey(), "12G")));
}).once();
- propStoreWatcher.signalCacheChangeEvent(eq(propCacheKey));
+ propStoreWatcher.signalCacheChangeEvent(eq(propStoreKey));
expectLastCall();
cacheMetrics.addLoadTime(anyLong());
@@ -481,18 +481,18 @@ public class ZooPropLoaderTest {
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
// prime cache
- var origProps = cache.get(propCacheKey);
+ var origProps = cache.get(propStoreKey);
assertNotNull(origProps);
assertEquals("7G", origProps.asMap().get(Property.TABLE_SPLIT_THRESHOLD.getKey()));
ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
// first call after refresh return original and schedules update
- var originalProps = cache.get(propCacheKey);
+ var originalProps = cache.get(propStoreKey);
assertNotNull(originalProps);
assertNotNull(originalProps.asMap().get(Property.TABLE_SPLIT_THRESHOLD.getKey()));
// refresh should have loaded updated value;
- var updatedProps = cache.get(propCacheKey);
+ var updatedProps = cache.get(propStoreKey);
log.debug("Updated props: {}", updatedProps == null ? "null" : updatedProps.print(true));
assertNotNull(updatedProps);
@@ -520,7 +520,7 @@ public class ZooPropLoaderTest {
Capture<Stat> stat = newCapture();
// first call loads cache
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
Stat s = stat.getValue();
s.setCtime(System.currentTimeMillis());
s.setMtime(System.currentTimeMillis());
@@ -534,7 +534,7 @@ public class ZooPropLoaderTest {
stat2.setMtime(System.currentTimeMillis());
stat2.setVersion(expectedVersion);
- expect(zrw.getStatus(propCacheKey.getPath())).andReturn(stat2).once();
+ expect(zrw.getStatus(propStoreKey.getPath())).andReturn(stat2).once();
cacheMetrics.addLoadTime(anyLong());
expectLastCall().times(1);
@@ -547,15 +547,15 @@ public class ZooPropLoaderTest {
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
// prime cache
- cache.get(propCacheKey);
+ cache.get(propStoreKey);
ticker.advance(30, TimeUnit.MINUTES);
- VersionedProperties vPropsRead = cache.get(propCacheKey);
+ VersionedProperties vPropsRead = cache.get(propStoreKey);
assertNotNull(vPropsRead);
- cache.get(propCacheKey);
+ cache.get(propStoreKey);
verify(mockProps);
}
@@ -572,7 +572,7 @@ public class ZooPropLoaderTest {
// first call loads cache
Capture<Stat> stat = newCapture();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), capture(stat))).andAnswer(() -> {
Stat s = stat.getValue();
s.setCtime(System.currentTimeMillis());
s.setMtime(System.currentTimeMillis());
@@ -584,13 +584,13 @@ public class ZooPropLoaderTest {
Stat expectedStat = new Stat();
expectedStat.setVersion(0);
- expect(zrw.getStatus(propCacheKey.getPath()))
+ expect(zrw.getStatus(propStoreKey.getPath()))
.andThrow(new KeeperException.NoNodeException("force no node exception")).once();
- propStoreWatcher.signalZkChangeEvent(eq(propCacheKey));
+ propStoreWatcher.signalZkChangeEvent(eq(propStoreKey));
expectLastCall().anyTimes();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(), anyObject()))
.andThrow(new KeeperException.NoNodeException("force no node exception")).once();
cacheMetrics.addLoadTime(anyLong());
@@ -606,13 +606,13 @@ public class ZooPropLoaderTest {
new PropCacheCaffeineImpl.Builder(loader, cacheMetrics).forTests(ticker).build();
// load cache
- log.debug("received: {}", cache.get(propCacheKey));
+ log.debug("received: {}", cache.get(propStoreKey));
ticker.advance(REFRESH_MIN + 1, TimeUnit.MINUTES);
- assertNotNull(cache.get(propCacheKey)); // returns current and queues async refresh
+ assertNotNull(cache.get(propStoreKey)); // returns current and queues async refresh
- assertNull(cache.get(propCacheKey)); // on exception, the loader should return null
+ assertNull(cache.get(propStoreKey)); // on exception, the loader should return null
}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java
index 745ecdcf7c..b9d5ff3182 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/store/impl/ZooPropStoreTest.java
@@ -85,17 +85,17 @@ public class ZooPropStoreTest {
@Test
public void create() throws Exception {
- var propCacheKey = TablePropKey.of(instanceId, TableId.of("propCacheKey"));
+ var propStoreKey = TablePropKey.of(instanceId, TableId.of("propStoreKey"));
Capture<byte[]> bytes = newCapture();
- expect(zrw.putPrivatePersistentData(eq(propCacheKey.getPath()), capture(bytes), anyObject()))
+ expect(zrw.putPrivatePersistentData(eq(propStoreKey.getPath()), capture(bytes), anyObject()))
.andReturn(true).once();
replay(context, zrw);
PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
- propStore.create(propCacheKey,
+ propStore.create(propStoreKey,
Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M"));
var decoded = propCodec.fromBytes(0, bytes.getValue());
@@ -114,22 +114,22 @@ public class ZooPropStoreTest {
@Test
public void getTest() throws Exception {
- var propCacheKey = TablePropKey.of(instanceId, TableId.of("propCacheKey"));
+ var propStoreKey = TablePropKey.of(instanceId, TableId.of("propStoreKey"));
var vProps = new VersionedProperties(Map.of(Property.TABLE_BLOOM_ENABLED.getKey(), "true"));
// expect one ZooKeeper call - subsequent calls should load from cache.
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
.andReturn(VersionedPropCodec.getDefault().toBytes(vProps)).once();
replay(context, zrw);
PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
- assertNotNull(propStore.get(propCacheKey)); // first call will fetch from ZooKeeper
- assertNotNull(propStore.get(propCacheKey)); // next call will fetch from cache.
+ assertNotNull(propStore.get(propStoreKey)); // first call will fetch from ZooKeeper
+ assertNotNull(propStore.get(propStoreKey)); // next call will fetch from cache.
- var p = propStore.get(propCacheKey);
+ var p = propStore.get(propStoreKey);
assertEquals("true", p.asMap().get(Property.TABLE_BLOOM_ENABLED.getKey()));
}
@@ -137,7 +137,7 @@ public class ZooPropStoreTest {
@Test
public void versionTest() throws Exception {
- var propCacheKey = TablePropKey.of(instanceId, TableId.of("table1"));
+ var propStoreKey = TablePropKey.of(instanceId, TableId.of("table1"));
Map<String,String> props =
Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M");
@@ -146,7 +146,7 @@ public class ZooPropStoreTest {
// force version mismatch between zk and props.
var expectedVersion = 99;
- expect(zrw.getData(eq(propCacheKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
+ expect(zrw.getData(eq(propStoreKey.getPath()), capture(propStoreWatcherCapture), capture(stat)))
.andAnswer(() -> {
Stat s = stat.getValue();
s.setCtime(System.currentTimeMillis());
@@ -159,7 +159,7 @@ public class ZooPropStoreTest {
replay(context, zrw);
PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
- var vProps = propStore.get(propCacheKey);
+ var vProps = propStore.get(propStoreKey);
assertNotNull(vProps);
assertEquals(expectedVersion, vProps.getDataVersion());
}
@@ -173,17 +173,17 @@ public class ZooPropStoreTest {
@Test
public void putAllTest() throws Exception {
- var propCacheKey = TablePropKey.of(instanceId, TableId.of("table1"));
+ var propStoreKey = TablePropKey.of(instanceId, TableId.of("table1"));
var initialProps = new VersionedProperties(0, Instant.now(),
Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M"));
// not cached - will load from ZooKeeper
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(Stat.class)))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(Stat.class)))
.andReturn(propCodec.toBytes(initialProps)).once();
Capture<byte[]> bytes = newCapture();
- expect(zrw.overwritePersistentData(eq(propCacheKey.getPath()), capture(bytes), eq(0)))
+ expect(zrw.overwritePersistentData(eq(propStoreKey.getPath()), capture(bytes), eq(0)))
.andAnswer(() -> {
var stored = propCodec.fromBytes(0, bytes.getValue());
assertEquals(3, stored.asMap().size());
@@ -203,7 +203,7 @@ public class ZooPropStoreTest {
Map<String,String> updateProps =
Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "4321", TABLE_SPLIT_THRESHOLD.getKey(), "123M");
- propStore.putAll(propCacheKey, updateProps);
+ propStore.putAll(propStoreKey, updateProps);
verify(zrw);
}
@@ -211,7 +211,7 @@ public class ZooPropStoreTest {
@Test
public void removeTest() throws Exception {
- var propCacheKey = TablePropKey.of(instanceId, TableId.of("table1"));
+ var propStoreKey = TablePropKey.of(instanceId, TableId.of("table1"));
var initialProps = new VersionedProperties(123, Instant.now(),
Map.of(TABLE_BULK_MAX_TABLETS.getKey(), "1234", TABLE_FILE_BLOCK_SIZE.getKey(), "512M"));
@@ -219,7 +219,7 @@ public class ZooPropStoreTest {
// not cached - will load from ZooKeeper
Capture<Stat> stat = newCapture();
- expect(zrw.getData(eq(propCacheKey.getPath()), capture(stat))).andAnswer(() -> {
+ expect(zrw.getData(eq(propStoreKey.getPath()), capture(stat))).andAnswer(() -> {
Stat s = stat.getValue();
s.setVersion(123);
stat.setValue(s);
@@ -227,7 +227,7 @@ public class ZooPropStoreTest {
}).once();
Capture<byte[]> bytes = newCapture();
- expect(zrw.overwritePersistentData(eq(propCacheKey.getPath()), capture(bytes), eq(123)))
+ expect(zrw.overwritePersistentData(eq(propStoreKey.getPath()), capture(bytes), eq(123)))
.andAnswer(() -> {
var stored = propCodec.fromBytes(124, bytes.getValue());
assertEquals(1, stored.asMap().size());
@@ -247,18 +247,18 @@ public class ZooPropStoreTest {
Set<String> deleteNames =
Set.of(TABLE_BULK_MAX_TABLETS.getKey(), TABLE_SPLIT_THRESHOLD.getKey());
- propStore.removeProperties(propCacheKey, deleteNames);
+ propStore.removeProperties(propStoreKey, deleteNames);
verify(zrw);
}
@Test
public void removeWithExceptionsTest() throws Exception {
- var propCacheKey = TablePropKey.of(instanceId, TableId.of("table1"));
+ var propStoreKey = TablePropKey.of(instanceId, TableId.of("table1"));
// return "bad data"
Capture<Stat> stat = newCapture();
- expect(zrw.getData(eq(propCacheKey.getPath()), capture(stat))).andAnswer(() -> {
+ expect(zrw.getData(eq(propStoreKey.getPath()), capture(stat))).andAnswer(() -> {
Stat s = stat.getValue();
s.setCtime(System.currentTimeMillis());
s.setMtime(System.currentTimeMillis());
@@ -269,9 +269,9 @@ public class ZooPropStoreTest {
}).once();
// mock throwing exceptions
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(Stat.class)))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(Stat.class)))
.andThrow(new KeeperException.NoNodeException("mock forced no node")).once();
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(Stat.class)))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(Stat.class)))
.andThrow(new InterruptedException("mock forced interrupt exception")).once();
replay(context, zrw);
@@ -283,13 +283,13 @@ public class ZooPropStoreTest {
// Parse error converted to IllegalStateException
assertThrows(IllegalStateException.class,
- () -> propStore.removeProperties(propCacheKey, deleteNames));
+ () -> propStore.removeProperties(propStoreKey, deleteNames));
// ZK exception converted to IllegalStateException
assertThrows(IllegalStateException.class,
- () -> propStore.removeProperties(propCacheKey, deleteNames));
+ () -> propStore.removeProperties(propStoreKey, deleteNames));
// InterruptException converted to IllegalStateException
assertThrows(IllegalStateException.class,
- () -> propStore.removeProperties(propCacheKey, deleteNames));
+ () -> propStore.removeProperties(propStoreKey, deleteNames));
}
@Test
@@ -376,28 +376,28 @@ public class ZooPropStoreTest {
@Test
public void deleteTest() throws Exception {
- var propCacheKey = TablePropKey.of(instanceId, TableId.of("propCacheKey"));
+ var propStoreKey = TablePropKey.of(instanceId, TableId.of("propStoreKey"));
var vProps = new VersionedProperties(Map.of(Property.TABLE_BLOOM_ENABLED.getKey(), "true"));
// expect first call to load cache.
- expect(zrw.getData(eq(propCacheKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
+ expect(zrw.getData(eq(propStoreKey.getPath()), anyObject(PropStoreWatcher.class), anyObject()))
.andReturn(VersionedPropCodec.getDefault().toBytes(vProps)).once();
- zrw.delete(eq(propCacheKey.getPath()));
+ zrw.delete(eq(propStoreKey.getPath()));
expectLastCall().once();
replay(context, zrw);
PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
- assertNotNull(propStore.get(propCacheKey)); // first call will fetch from ZooKeeper
- assertNotNull(propStore.get(propCacheKey)); // next call will fetch from cache.
- var p = propStore.get(propCacheKey);
+ assertNotNull(propStore.get(propStoreKey)); // first call will fetch from ZooKeeper
+ assertNotNull(propStore.get(propStoreKey)); // next call will fetch from cache.
+ var p = propStore.get(propStoreKey);
assertEquals("true", p.asMap().get(Property.TABLE_BLOOM_ENABLED.getKey()));
- propStore.delete(propCacheKey);
+ propStore.delete(propStoreKey);
Thread.sleep(50);
}
}
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java
index 2f0b9034dd..27b1094b75 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/store/PropCacheCaffeineImplZkIT.java
@@ -151,8 +151,8 @@ public class PropCacheCaffeineImplZkIT {
VersionedProperties vProps = new VersionedProperties(props);
// directly create prop node - simulate existing properties.
- var propCacheKey = TablePropKey.of(INSTANCE_ID, tIdA);
- var created = zrw.putPersistentData(propCacheKey.getPath(),
+ var propStoreKey = TablePropKey.of(INSTANCE_ID, tIdA);
+ var created = zrw.putPersistentData(propStoreKey.getPath(),
VersionedPropCodec.getDefault().toBytes(vProps), ZooUtil.NodeExistsPolicy.FAIL);
assertTrue(created, "expected properties to be created");
@@ -168,7 +168,7 @@ public class PropCacheCaffeineImplZkIT {
PropCacheCaffeineImpl cache =
new PropCacheCaffeineImpl.Builder(propLoader, cacheMetrics).build();
- VersionedProperties readProps = cache.get(propCacheKey);
+ VersionedProperties readProps = cache.get(propStoreKey);
if (readProps == null) {
fail("Received null for versioned properties");
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java
index c6f2d4b92d..a107498b24 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/store/PropStoreZooKeeperIT.java
@@ -43,9 +43,9 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil;
import org.apache.accumulo.server.ServerContext;
import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
import org.apache.accumulo.server.conf.store.PropChangeListener;
import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.accumulo.server.conf.store.TablePropKey;
import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
@@ -383,22 +383,22 @@ public class PropStoreZooKeeperIT {
private static class TestChangeListener implements PropChangeListener {
- private final Map<PropCacheKey<?>,Integer> changeCounts = new ConcurrentHashMap<>();
- private final Map<PropCacheKey<?>,Integer> deleteCounts = new ConcurrentHashMap<>();
+ private final Map<PropStoreKey<?>,Integer> changeCounts = new ConcurrentHashMap<>();
+ private final Map<PropStoreKey<?>,Integer> deleteCounts = new ConcurrentHashMap<>();
@Override
- public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
- changeCounts.merge(propCacheKey, 1, Integer::sum);
+ public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
+ changeCounts.merge(propStoreKey, 1, Integer::sum);
}
@Override
- public void cacheChangeEvent(PropCacheKey<?> propCacheKey) {
- changeCounts.merge(propCacheKey, 1, Integer::sum);
+ public void cacheChangeEvent(PropStoreKey<?> propStoreKey) {
+ changeCounts.merge(propStoreKey, 1, Integer::sum);
}
@Override
- public void deleteEvent(PropCacheKey<?> propCacheKey) {
- deleteCounts.merge(propCacheKey, 1, Integer::sum);
+ public void deleteEvent(PropStoreKey<?> propStoreKey) {
+ deleteCounts.merge(propStoreKey, 1, Integer::sum);
}
@Override
@@ -406,11 +406,11 @@ public class PropStoreZooKeeperIT {
}
- public Map<PropCacheKey<?>,Integer> getChangeCounts() {
+ public Map<PropStoreKey<?>,Integer> getChangeCounts() {
return changeCounts;
}
- public Map<PropCacheKey<?>,Integer> getDeleteCounts() {
+ public Map<PropStoreKey<?>,Integer> getDeleteCounts() {
return deleteCounts;
}
}
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java b/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java
index 47e59cc76a..cccc8c1f40 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/store/ZooBasedConfigIT.java
@@ -51,9 +51,9 @@ import org.apache.accumulo.server.conf.NamespaceConfiguration;
import org.apache.accumulo.server.conf.SystemConfiguration;
import org.apache.accumulo.server.conf.ZooBasedConfiguration;
import org.apache.accumulo.server.conf.store.NamespacePropKey;
-import org.apache.accumulo.server.conf.store.PropCacheKey;
import org.apache.accumulo.server.conf.store.PropChangeListener;
import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
import org.apache.accumulo.server.conf.store.SystemPropKey;
import org.apache.accumulo.server.conf.store.TablePropKey;
import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
@@ -339,20 +339,20 @@ public class ZooBasedConfigIT {
}
@Override
- public void zkChangeEvent(PropCacheKey<?> propCacheKey) {
- log.debug("Received zkChangeEvent for {}", propCacheKey);
+ public void zkChangeEvent(PropStoreKey<?> propStoreKey) {
+ log.debug("Received zkChangeEvent for {}", propStoreKey);
zkChangeCount.incrementAndGet();
}
@Override
- public void cacheChangeEvent(PropCacheKey<?> propCacheKey) {
- log.debug("Received cacheChangeEvent for {}", propCacheKey);
+ public void cacheChangeEvent(PropStoreKey<?> propStoreKey) {
+ log.debug("Received cacheChangeEvent for {}", propStoreKey);
cacheChangeCount.incrementAndGet();
}
@Override
- public void deleteEvent(PropCacheKey<?> propCacheKey) {
- log.debug("Received deleteEvent for: {}", propCacheKey);
+ public void deleteEvent(PropStoreKey<?> propStoreKey) {
+ log.debug("Received deleteEvent for: {}", propStoreKey);
deleteCount.incrementAndGet();
}