You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2023/07/18 16:02:46 UTC

[accumulo] branch 2.1 updated: Avoid caching a snapshot of custom props (#3588)

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

kturner pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new c679b04313 Avoid caching a snapshot of custom props (#3588)
c679b04313 is described below

commit c679b04313ae4117f290960cca637dd7bc78baf3
Author: Keith Turner <kt...@apache.org>
AuthorDate: Tue Jul 18 12:02:39 2023 -0400

    Avoid caching a snapshot of custom props (#3588)
    
    ConfigurationImpl which is provided to plugin impl cached a snapshot of
    custom props indefinitely.  However other props were not cached in this
    way.  This change removes the indefinited caching of custom props.
---
 .../accumulo/core/util/ConfigurationImpl.java      | 24 +++++------
 .../accumulo/core/util/ConfigurationImplTest.java  | 49 ++++++++++++++++++++++
 .../accumulo/server/ServiceEnvironmentImpl.java    | 13 ++++--
 .../server/ServiceEnvironmentImplTest.java         | 43 +++++++++----------
 4 files changed, 90 insertions(+), 39 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/util/ConfigurationImpl.java b/core/src/main/java/org/apache/accumulo/core/util/ConfigurationImpl.java
index 6bb2015618..e80cb998ad 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ConfigurationImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ConfigurationImpl.java
@@ -38,11 +38,15 @@ import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
 public class ConfigurationImpl implements Configuration {
 
   private final AccumuloConfiguration acfg;
-  private Map<String,String> customProps;
-  private Map<String,String> tableCustomProps;
+  private final AccumuloConfiguration.Deriver<Map<String,String>> tableCustomDeriver;
+  private final AccumuloConfiguration.Deriver<Map<String,String>> customDeriver;
 
   public ConfigurationImpl(AccumuloConfiguration acfg) {
     this.acfg = acfg;
+    this.customDeriver =
+        acfg.newDeriver(aconf -> buildCustom(aconf, Property.GENERAL_ARBITRARY_PROP_PREFIX));
+    this.tableCustomDeriver =
+        acfg.newDeriver(aconf -> buildCustom(aconf, Property.TABLE_ARBITRARY_PROP_PREFIX));
   }
 
   @Override
@@ -83,11 +87,7 @@ public class ConfigurationImpl implements Configuration {
 
   @Override
   public Map<String,String> getCustom() {
-    if (customProps == null) {
-      customProps = buildCustom(Property.GENERAL_ARBITRARY_PROP_PREFIX);
-    }
-
-    return customProps;
+    return customDeriver.derive();
   }
 
   @Override
@@ -97,11 +97,7 @@ public class ConfigurationImpl implements Configuration {
 
   @Override
   public Map<String,String> getTableCustom() {
-    if (tableCustomProps == null) {
-      tableCustomProps = buildCustom(Property.TABLE_ARBITRARY_PROP_PREFIX);
-    }
-
-    return tableCustomProps;
+    return tableCustomDeriver.derive();
   }
 
   @Override
@@ -109,8 +105,8 @@ public class ConfigurationImpl implements Configuration {
     return getTableCustom().get(keySuffix);
   }
 
-  private Map<String,String> buildCustom(Property customPrefix) {
-    return acfg.getAllPropertiesWithPrefix(customPrefix).entrySet().stream().collect(
+  private static Map<String,String> buildCustom(AccumuloConfiguration conf, Property customPrefix) {
+    return conf.getAllPropertiesWithPrefix(customPrefix).entrySet().stream().collect(
         Collectors.toUnmodifiableMap(e -> e.getKey().substring(customPrefix.getKey().length()),
             Entry::getValue));
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/util/ConfigurationImplTest.java b/core/src/test/java/org/apache/accumulo/core/util/ConfigurationImplTest.java
new file mode 100644
index 0000000000..fba1bee4a0
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/ConfigurationImplTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.core.util;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+import java.util.Map;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.junit.jupiter.api.Test;
+
+public class ConfigurationImplTest {
+  @Test
+  public void testCustomProps() {
+    ConfigurationCopy conf = new ConfigurationCopy();
+    conf.set("table.custom.p1", "v1");
+    conf.set("table.custom.p2", "v2");
+    conf.set("general.custom.p3", "v3");
+    conf.set("general.custom.p4", "v4");
+
+    var confImp = new ConfigurationImpl(conf);
+
+    assertEquals(Map.of("p3", "v3", "p4", "v4"), confImp.getCustom());
+    assertEquals(Map.of("p1", "v1", "p2", "v2"), confImp.getTableCustom());
+
+    assertEquals("v3", confImp.getCustom("p3"));
+    assertEquals("v1", confImp.getTableCustom("p1"));
+
+    assertNull(confImp.getCustom("p1"));
+    assertNull(confImp.getTableCustom("p3"));
+  }
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
index fc2161e657..0f68a11d48 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
@@ -19,8 +19,7 @@
 package org.apache.accumulo.server;
 
 import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -29,14 +28,20 @@ import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.util.ConfigurationImpl;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+
 public class ServiceEnvironmentImpl implements ServiceEnvironment {
 
   private final ServerContext context;
   private final Configuration conf;
-  private final Map<TableId,Configuration> tableConfigs = new ConcurrentHashMap<>();
+  private final Cache<TableId,Configuration> tableConfigs;
 
   public ServiceEnvironmentImpl(ServerContext context) {
     this.context = context;
+    // For a long-lived instance of this object, avoid keeping references around to tables that may
+    // have been deleted.
+    this.tableConfigs = Caffeine.newBuilder().expireAfterAccess(10, TimeUnit.MINUTES).build();
     this.conf = new ConfigurationImpl(this.context.getConfiguration());
   }
 
@@ -47,7 +52,7 @@ public class ServiceEnvironmentImpl implements ServiceEnvironment {
 
   @Override
   public Configuration getConfiguration(TableId tableId) {
-    return tableConfigs.computeIfAbsent(tableId,
+    return tableConfigs.get(tableId,
         tid -> new ConfigurationImpl(context.getTableConfiguration(tid)));
   }
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServiceEnvironmentImplTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServiceEnvironmentImplTest.java
index 4ad6766141..86e696abbe 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/ServiceEnvironmentImplTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/ServiceEnvironmentImplTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.server;
 
+import static org.easymock.EasyMock.anyObject;
 import static org.easymock.EasyMock.createMock;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.replay;
@@ -28,52 +29,52 @@ import java.util.Map;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
 public class ServiceEnvironmentImplTest {
-  private ServerContext context;
-  private AccumuloConfiguration acfg;
-  private ServiceEnvironmentImpl serviceEnvironment;
-
-  @BeforeEach
-  public void setUp() {
-    context = createMock(ServerContext.class);
-    acfg = createMock(AccumuloConfiguration.class);
-    expect(context.getConfiguration()).andReturn(acfg);
-    replay(context);
-    serviceEnvironment = new ServiceEnvironmentImpl(context);
-  }
-
-  @AfterEach
-  public void verifyMocks() {
-    verify(context, acfg);
-  }
 
   @Test
   public void getWithRecognizedPrefixTest() {
+
     String prefix = Property.RPC_PREFIX.getKey();
     Map<String,String> expectedPropertyMap = Map.of("rpc.javax.net.ssl.keyStoreType", "jks");
+
+    ServerContext context = createMock(ServerContext.class);
+    AccumuloConfiguration acfg = createMock(AccumuloConfiguration.class);
+    expect(acfg.newDeriver(anyObject())).andReturn(Map::of).anyTimes();
     expect(acfg.getAllPropertiesWithPrefix(Property.RPC_PREFIX)).andReturn(expectedPropertyMap);
-    replay(acfg);
+    expect(context.getConfiguration()).andReturn(acfg);
+    replay(context, acfg);
+
+    var serviceEnvironment = new ServiceEnvironmentImpl(context);
 
     Map<String,String> returnedProperties =
         serviceEnvironment.getConfiguration().getWithPrefix(prefix);
 
     assertEquals(expectedPropertyMap, returnedProperties);
+
+    verify(context, acfg);
   }
 
   @Test
   public void getWithUnrecognizedPrefixTest() {
     String prefix = "a.b";
     Map<String,String> expectedPropertyMap = Map.of("a.b.favorite.license", "apache");
+
+    ServerContext context = createMock(ServerContext.class);
+    AccumuloConfiguration acfg = createMock(AccumuloConfiguration.class);
+    expect(acfg.newDeriver(anyObject())).andReturn(Map::of).anyTimes();
     expect(acfg.spliterator()).andReturn(expectedPropertyMap.entrySet().spliterator());
-    replay(acfg);
+    expect(context.getConfiguration()).andReturn(acfg);
+    replay(context, acfg);
+
+    var serviceEnvironment = new ServiceEnvironmentImpl(context);
 
     Map<String,String> returnedProperties =
         serviceEnvironment.getConfiguration().getWithPrefix(prefix);
 
     assertEquals(expectedPropertyMap, returnedProperties);
+
+    verify(context, acfg);
   }
 }