keith-turner commented on code in PR #3180:
URL: https://github.com/apache/accumulo/pull/3180#discussion_r1096872490
##
server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java:
##
@@ -353,6 +362,38 @@ public short getDefaultReplication(Path path) {
return getFileSystemByPath(path).getDefaultReplication(path);
}
+ /**
+ * The Hadoop Configuration object does not currently allow for duplicate
properties to be set in
+ * a single Configuration for different FileSystem URIs. Here we will look
for properties in the
+ * Accumulo configuration of the form:
+ *
+ *
+ * general.custom.volume-uri.hdfs-property
+ *
+ *
+ * We will use these properties to return a new Configuration object that
can be used with the
+ * FileSystem URI.
+ *
+ * @param conf AccumuloConfiguration object
+ * @param hadoopConf Hadoop Configuration object
+ * @param filesystemURI Volume Filesystem URI
+ * @return Hadoop Configuration with custom overrides for this FileSystem
+ */
+ private static Configuration
getVolumeManagerConfiguration(AccumuloConfiguration conf,
+ final Configuration hadoopConf, final String filesystemURI) {
+final Configuration volumeConfig = new Configuration(hadoopConf);
+final Map customProps =
+
conf.getAllPropertiesWithPrefixStripped(Property.GENERAL_ARBITRARY_PROP_PREFIX);
+customProps.forEach((key, value) -> {
+ if (key.startsWith(filesystemURI)) {
+String property = key.substring(filesystemURI.length() + 1);
+log.debug("Overriding property {} to {} for volume {}", property,
value, filesystemURI);
Review Comment:
Really like the logging. Does this logging only happen once per volume on a
server? If its only once, maybe could promote it to info.
##
server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java:
##
@@ -104,4 +111,88 @@ public Scope getChooserScope() {
assertThrows(RuntimeException.class, () -> vm.choose(chooserEnv,
volumes));
}
}
+
+ @Test
+ public void testConfigurationOverrides() throws Exception {
+
+final String vol1 = "file://127.0.0.1/vol1/";
+final String vol2 = "file://localhost/vol2/";
+final String vol3 = "hdfs://127.0.0.1/accumulo";
+final String vol4 = "hdfs://localhost/accumulo";
+
+ConfigurationCopy conf = new ConfigurationCopy();
+conf.set(Property.INSTANCE_VOLUMES, String.join(",", vol1, vol2, vol3,
vol4));
+conf.set(Property.GENERAL_VOLUME_CHOOSER,
Property.GENERAL_VOLUME_CHOOSER.getDefaultValue());
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol1 + "."
++ HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, "10");
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol1 + "."
Review Comment:
Could set a third override on one volume that no others set to ensure there
is no interference for this case. For example could set a third blocksize prop
for only vol1.
##
server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java:
##
@@ -104,4 +111,88 @@ public Scope getChooserScope() {
assertThrows(RuntimeException.class, () -> vm.choose(chooserEnv,
volumes));
}
}
+
+ @Test
+ public void testConfigurationOverrides() throws Exception {
+
+final String vol1 = "file://127.0.0.1/vol1/";
+final String vol2 = "file://localhost/vol2/";
+final String vol3 = "hdfs://127.0.0.1/accumulo";
+final String vol4 = "hdfs://localhost/accumulo";
+
+ConfigurationCopy conf = new ConfigurationCopy();
+conf.set(Property.INSTANCE_VOLUMES, String.join(",", vol1, vol2, vol3,
vol4));
+conf.set(Property.GENERAL_VOLUME_CHOOSER,
Property.GENERAL_VOLUME_CHOOSER.getDefaultValue());
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol1 + "."
++ HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, "10");
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol1 + "."
++ HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS, "true");
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol2 + "."
++ HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, "20");
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol2 + "."
++ HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS, "false");
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol3 + "."
++ HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, "30");
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol3 + "."
++ HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS, "TRUE");
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol4 + "."
++ HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, "40");
+conf.set(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + vol4 + "."
++ HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS, "FALSE");
+