You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by do...@apache.org on 2022/07/06 13:23:44 UTC

[accumulo] branch main updated: Refactor common code into PropUtil (#2796)

This is an automated email from the ASF dual-hosted git repository.

domgarguilo 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 70a27ecefe Refactor common code into PropUtil (#2796)
70a27ecefe is described below

commit 70a27ecefe2fc28c54f8731a55dd654e38fdb7f4
Author: Dom G <do...@gmail.com>
AuthorDate: Wed Jul 6 09:23:38 2022 -0400

    Refactor common code into PropUtil (#2796)
    
    * Refactor out common code into parent class
    * Remove Namespace,Table - PropUtil classes
---
 .../org/apache/accumulo/server/ServerContext.java  | 17 ++----
 .../accumulo/server/tables/TableManager.java       |  4 +-
 .../accumulo/server/util/NamespacePropUtil.java    | 61 ----------------------
 .../org/apache/accumulo/server/util/PropUtil.java  | 32 ++++++++++--
 .../apache/accumulo/server/util/TablePropUtil.java | 60 ---------------------
 .../manager/ManagerClientServiceHandler.java       | 15 ++++--
 .../manager/tableOps/create/PopulateZookeeper.java |  4 +-
 .../create/PopulateZookeeperWithNamespace.java     |  4 +-
 .../tableImport/ImportPopulateZookeeper.java       |  4 +-
 .../accumulo/manager/upgrade/Upgrader9to10.java    | 21 +++++---
 .../accumulo/test/ZooKeeperPropertiesIT.java       | 17 +++---
 11 files changed, 77 insertions(+), 162 deletions(-)

diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index 800251b9d1..17dca56d47 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -78,8 +78,7 @@ import org.apache.accumulo.server.security.SecurityUtil;
 import org.apache.accumulo.server.security.delegation.AuthenticationTokenSecretManager;
 import org.apache.accumulo.server.tables.TableManager;
 import org.apache.accumulo.server.tablets.UniqueNameAllocator;
-import org.apache.accumulo.server.util.NamespacePropUtil;
-import org.apache.accumulo.server.util.TablePropUtil;
+import org.apache.accumulo.server.util.PropUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.zookeeper.KeeperException;
@@ -107,8 +106,7 @@ public class ServerContext extends ClientContext {
   private final Supplier<CryptoService> cryptoService;
   private final Supplier<ScheduledThreadPoolExecutor> sharedScheduledThreadPool;
   private final Supplier<AuditedSecurityOperation> securityOperation;
-  private final Supplier<TablePropUtil> tablePropUtilSupplier;
-  private final Supplier<NamespacePropUtil> namespacePropUtilSupplier;
+  private final Supplier<PropUtil> propUtilSupplier;
 
   public ServerContext(SiteConfiguration siteConfig) {
     this(new ServerInfo(siteConfig));
@@ -136,8 +134,7 @@ public class ServerContext extends ClientContext {
     securityOperation =
         memoize(() -> new AuditedSecurityOperation(this, SecurityOperation.getAuthorizor(this),
             SecurityOperation.getAuthenticator(this), SecurityOperation.getPermHandler(this)));
-    tablePropUtilSupplier = memoize(() -> new TablePropUtil(this));
-    namespacePropUtilSupplier = memoize(() -> new NamespacePropUtil(this));
+    propUtilSupplier = memoize(() -> new PropUtil(this));
   }
 
   /**
@@ -456,12 +453,8 @@ public class ServerContext extends ClientContext {
     return securityOperation.get();
   }
 
-  public TablePropUtil tablePropUtil() {
-    return tablePropUtilSupplier.get();
-  }
-
-  public NamespacePropUtil namespacePropUtil() {
-    return namespacePropUtilSupplier.get();
+  public PropUtil propUtil() {
+    return propUtilSupplier.get();
   }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
index bff73f1182..beb70626db 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
@@ -225,8 +225,8 @@ public class TableManager {
         + Constants.ZTABLE_CONF;
     zoo.recursiveCopyPersistentOverwrite(srcTablePath, newTablePath);
 
-    context.tablePropUtil().setProperties(tableId, propertiesToSet);
-    context.tablePropUtil().removeProperties(tableId, propertiesToExclude);
+    context.propUtil().setProperties(TablePropKey.of(context, tableId), propertiesToSet);
+    context.propUtil().removeProperties(TablePropKey.of(context, tableId), propertiesToExclude);
 
     updateTableStateCache(tableId);
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
deleted file mode 100644
index 3e1dea5911..0000000000
--- a/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.server.util;
-
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.NamespaceId;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.store.NamespacePropKey;
-
-public class NamespacePropUtil extends PropUtil<NamespaceId> {
-
-  public NamespacePropUtil(ServerContext context) {
-    super(context);
-  }
-
-  /**
-   * Helper method to set provided properties for the provided namespace.
-   *
-   * @throws IllegalStateException
-   *           if an underlying exception (KeeperException, InterruptException) or other failure to
-   *           read properties from the cache / backend store
-   * @throws IllegalArgumentException
-   *           if a provided property is not valid
-   */
-  @Override
-  public void setProperties(NamespaceId namespaceId, Map<String,String> properties) {
-    for (Map.Entry<String,String> prop : properties.entrySet()) {
-      if (!Property.isTablePropertyValid(prop.getKey(), prop.getValue())) {
-        throw new IllegalArgumentException("Invalid property for namespace: " + namespaceId
-            + " name: " + prop.getKey() + ", value: " + prop.getValue());
-      }
-    }
-    context.getPropStore().putAll(NamespacePropKey.of(context, namespaceId), properties);
-  }
-
-  @Override
-  public void removeProperties(NamespaceId namespaceId, Collection<String> propertyNames) {
-    context.getPropStore().removeProperties(NamespacePropKey.of(context, namespaceId),
-        propertyNames);
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/PropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/PropUtil.java
index 763c547415..5e6846929f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/PropUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/PropUtil.java
@@ -21,19 +21,41 @@ package org.apache.accumulo.server.util;
 import java.util.Collection;
 import java.util.Map;
 
-import org.apache.accumulo.core.data.AbstractId;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
 
-public abstract class PropUtil<T extends AbstractId<T>> {
+public class PropUtil {
 
   protected final ServerContext context;
 
-  protected PropUtil(ServerContext context) {
+  public PropUtil(ServerContext context) {
     this.context = context;
   }
 
-  public abstract void setProperties(final T id, final Map<String,String> props);
+  /**
+   * Method to set provided properties for the provided AbstractId.
+   *
+   * @throws IllegalStateException
+   *           if an underlying exception (KeeperException, InterruptException) or other failure to
+   *           read properties from the cache / backend store
+   * @throws IllegalArgumentException
+   *           if a provided property is not valid
+   */
+  public void setProperties(final PropStoreKey<?> propStoreKey,
+      final Map<String,String> properties) {
+    for (Map.Entry<String,String> prop : properties.entrySet()) {
+      if (!Property.isTablePropertyValid(prop.getKey(), prop.getValue())) {
+        throw new IllegalArgumentException("Invalid property for : " + propStoreKey + " name: "
+            + prop.getKey() + ", value: " + prop.getValue());
+      }
+    }
+    context.getPropStore().putAll(propStoreKey, properties);
+  }
 
-  public abstract void removeProperties(final T id, final Collection<String> propertyNames);
+  public void removeProperties(final PropStoreKey<?> propStoreKey,
+      final Collection<String> propertyNames) {
+    context.getPropStore().removeProperties(propStoreKey, propertyNames);
+  }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
deleted file mode 100644
index c12698ae2e..0000000000
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.server.util;
-
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.store.TablePropKey;
-
-public class TablePropUtil extends PropUtil<TableId> {
-
-  public TablePropUtil(ServerContext context) {
-    super(context);
-  }
-
-  /**
-   * Helper method to set provided properties for the provided table.
-   *
-   * @throws IllegalStateException
-   *           if an underlying exception (KeeperException, InterruptException) or other failure to
-   *           read properties from the cache / backend store
-   * @throws IllegalArgumentException
-   *           if a provided property is not valid
-   */
-  @Override
-  public void setProperties(TableId tableId, Map<String,String> properties) {
-    for (Map.Entry<String,String> prop : properties.entrySet()) {
-      if (!Property.isTablePropertyValid(prop.getKey(), prop.getValue())) {
-        throw new IllegalArgumentException("Invalid property for table: " + tableId + " name: "
-            + prop.getKey() + ", value: " + prop.getValue());
-      }
-    }
-    context.getPropStore().putAll(TablePropKey.of(context, tableId), properties);
-  }
-
-  @Override
-  public void removeProperties(TableId tableId, Collection<String> propertyNames) {
-    context.getPropStore().removeProperties(TablePropKey.of(context, tableId), propertyNames);
-  }
-
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
index 75fea62fe6..60fd7c303d 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
@@ -81,6 +81,8 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.manager.tableOps.TraceRepo;
 import org.apache.accumulo.manager.tserverOps.ShutdownTServer;
 import org.apache.accumulo.server.client.ClientServiceHandler;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.security.delegation.AuthenticationTokenSecretManager;
@@ -399,10 +401,11 @@ public class ManagerClientServiceHandler implements ManagerClientService.Iface {
 
     try {
       if (value == null) {
-        manager.getContext().namespacePropUtil().removeProperties(namespaceId, List.of(property));
+        manager.getContext().propUtil().removeProperties(
+            NamespacePropKey.of(manager.getContext(), namespaceId), List.of(property));
       } else {
-        manager.getContext().namespacePropUtil().setProperties(namespaceId,
-            Map.of(property, value));
+        manager.getContext().propUtil().setProperties(
+            NamespacePropKey.of(manager.getContext(), namespaceId), Map.of(property, value));
       }
     } catch (IllegalStateException ex) {
       // race condition on delete... namespace no longer exists? An undelying ZooKeeper.NoNode
@@ -423,9 +426,11 @@ public class ManagerClientServiceHandler implements ManagerClientService.Iface {
 
     try {
       if (value == null || value.isEmpty()) {
-        manager.getContext().tablePropUtil().removeProperties(tableId, List.of(property));
+        manager.getContext().propUtil()
+            .removeProperties(TablePropKey.of(manager.getContext(), tableId), List.of(property));
       } else {
-        manager.getContext().tablePropUtil().setProperties(tableId, Map.of(property, value));
+        manager.getContext().propUtil()
+            .setProperties(TablePropKey.of(manager.getContext(), tableId), Map.of(property, value));
       }
     } catch (IllegalStateException ex) {
       log.warn("Invalid table property, tried to set: tableId: " + tableId.canonical() + " to: "
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
index 3fd369a4ac..2d7fcba6dd 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.TableInfo;
 import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.accumulo.server.conf.store.TablePropKey;
 
 class PopulateZookeeper extends ManagerRepo {
 
@@ -57,7 +58,8 @@ class PopulateZookeeper extends ManagerRepo {
           tableInfo.getTableName());
 
       try {
-        manager.getContext().tablePropUtil().setProperties(tableInfo.getTableId(), tableInfo.props);
+        manager.getContext().propUtil().setProperties(
+            TablePropKey.of(manager.getContext(), tableInfo.getTableId()), tableInfo.props);
       } catch (IllegalStateException ex) {
         throw new ThriftTableOperationException(null, tableInfo.getTableName(),
             TableOperation.CREATE, TableOperationExceptionType.OTHER,
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
index 68b898e4ee..07eeaee993 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
@@ -24,6 +24,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
 import org.apache.accumulo.server.tables.TableManager;
 
 class PopulateZookeeperWithNamespace extends ManagerRepo {
@@ -53,7 +54,8 @@ class PopulateZookeeperWithNamespace extends ManagerRepo {
       TableManager.prepareNewNamespaceState(manager.getContext(), namespaceInfo.namespaceId,
           namespaceInfo.namespaceName, NodeExistsPolicy.OVERWRITE);
 
-      manager.getContext().namespacePropUtil().setProperties(namespaceInfo.namespaceId,
+      manager.getContext().propUtil().setProperties(
+          NamespacePropKey.of(manager.getContext(), namespaceInfo.namespaceId),
           namespaceInfo.props);
 
       manager.getContext().clearTableListCache();
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java
index d1f4ee1fd1..9e6cb3bd9c 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java
@@ -32,6 +32,7 @@ import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
+import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -88,7 +89,8 @@ class ImportPopulateZookeeper extends ManagerRepo {
     VolumeManager volMan = env.getVolumeManager();
 
     try {
-      env.getContext().tablePropUtil().setProperties(tableInfo.tableId, getExportedProps(volMan));
+      env.getContext().propUtil().setProperties(
+          TablePropKey.of(env.getContext(), tableInfo.tableId), getExportedProps(volMan));
     } catch (IllegalStateException ex) {
       throw new AcceptableThriftTableOperationException(tableInfo.tableId.canonical(),
           tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java
index 79846de5c9..3fb9ee3585 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java
@@ -34,6 +34,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.function.BiConsumer;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -73,6 +74,7 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.apache.accumulo.server.conf.util.ConfigPropertyUpgrader;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.gc.AllVolumesDirectory;
@@ -158,16 +160,19 @@ public class Upgrader9to10 implements Upgrader {
    */
   private void setMetaTableProps(ServerContext context) {
     try {
+      // sets the compaction dispatcher props for the given table and service name
+      BiConsumer<TableId,String> setDispatcherProps =
+          (TableId tableId, String dispatcherService) -> {
+            var dispatcherPropsMap = Map.of(Property.TABLE_COMPACTION_DISPATCHER.getKey(),
+                SimpleCompactionDispatcher.class.getName(),
+                Property.TABLE_COMPACTION_DISPATCHER_OPTS.getKey() + "service", dispatcherService);
+            context.propUtil().setProperties(TablePropKey.of(context, tableId), dispatcherPropsMap);
+          };
+
       // root compaction props
-      context.tablePropUtil().setProperties(RootTable.ID,
-          Map.of(Property.TABLE_COMPACTION_DISPATCHER.getKey(),
-              SimpleCompactionDispatcher.class.getName(),
-              Property.TABLE_COMPACTION_DISPATCHER_OPTS.getKey() + "service", "root"));
+      setDispatcherProps.accept(RootTable.ID, "root");
       // metadata compaction props
-      context.tablePropUtil().setProperties(MetadataTable.ID,
-          Map.of(Property.TABLE_COMPACTION_DISPATCHER.getKey(),
-              SimpleCompactionDispatcher.class.getName(),
-              Property.TABLE_COMPACTION_DISPATCHER_OPTS.getKey() + "service", "meta"));
+      setDispatcherProps.accept(MetadataTable.ID, "meta");
     } catch (IllegalStateException ex) {
       throw new RuntimeException("Unable to set system table properties", ex);
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/ZooKeeperPropertiesIT.java b/test/src/main/java/org/apache/accumulo/test/ZooKeeperPropertiesIT.java
index 7449bcc465..87d34cd444 100644
--- a/test/src/main/java/org/apache/accumulo/test/ZooKeeperPropertiesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ZooKeeperPropertiesIT.java
@@ -38,6 +38,8 @@ import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
 
@@ -68,7 +70,8 @@ public class ZooKeeperPropertiesIT extends AccumuloClusterHarness {
       Map<String,String> properties = client.tableOperations().getConfiguration(tableName);
       assertEquals("false", properties.get(Property.TABLE_BLOOM_ENABLED.getKey()));
 
-      context.tablePropUtil().setProperties(TableId.of(tid),
+      final TablePropKey tablePropKey = TablePropKey.of(context, TableId.of(tid));
+      context.propUtil().setProperties(tablePropKey,
           Map.of(Property.TABLE_BLOOM_ENABLED.getKey(), "true"));
 
       // add a sleep to give the property change time to propagate
@@ -82,7 +85,7 @@ public class ZooKeeperPropertiesIT extends AccumuloClusterHarness {
         properties = client.tableOperations().getConfiguration(tableName);
       }
 
-      context.tablePropUtil().removeProperties(TableId.of(tid),
+      context.propUtil().removeProperties(tablePropKey,
           List.of(Property.TABLE_BLOOM_ENABLED.getKey()));
 
       properties = client.tableOperations().getConfiguration(tableName);
@@ -97,7 +100,7 @@ public class ZooKeeperPropertiesIT extends AccumuloClusterHarness {
 
       // Add invalid property
       assertThrows(IllegalArgumentException.class,
-          () -> context.tablePropUtil().setProperties(TableId.of(tid),
+          () -> context.propUtil().setProperties(tablePropKey,
               Map.of("NOT_A_PROPERTY", "not_a_value")),
           "Expected IllegalArgumentException to be thrown.");
     }
@@ -119,7 +122,9 @@ public class ZooKeeperPropertiesIT extends AccumuloClusterHarness {
       Map<String,String> properties = client.namespaceOperations().getConfiguration(namespace);
       assertEquals("15", properties.get(Property.TABLE_FILE_MAX.getKey()));
 
-      context.namespacePropUtil().setProperties(NamespaceId.of(nid),
+      final NamespaceId namespaceId = NamespaceId.of(nid);
+      final NamespacePropKey namespacePropKey = NamespacePropKey.of(context, namespaceId);
+      context.propUtil().setProperties(namespacePropKey,
           Map.of(Property.TABLE_FILE_MAX.getKey(), "31"));
 
       // add a sleep to give the property change time to propagate
@@ -133,7 +138,7 @@ public class ZooKeeperPropertiesIT extends AccumuloClusterHarness {
         properties = client.namespaceOperations().getConfiguration(namespace);
       }
 
-      context.namespacePropUtil().removeProperties(NamespaceId.of(nid),
+      context.propUtil().removeProperties(namespacePropKey,
           List.of(Property.TABLE_FILE_MAX.getKey()));
 
       properties = client.namespaceOperations().getConfiguration(namespace);
@@ -148,7 +153,7 @@ public class ZooKeeperPropertiesIT extends AccumuloClusterHarness {
 
       // Add invalid property
       assertThrows(IllegalArgumentException.class,
-          () -> context.namespacePropUtil().setProperties(NamespaceId.of(nid),
+          () -> context.propUtil().setProperties(namespacePropKey,
               Map.of("NOT_A_PROPERTY", "not_a_value")),
           "Expected IllegalArgumentException to be thrown.");
     }