ctubbsii commented on code in PR #3180:
URL: https://github.com/apache/accumulo/pull/3180#discussion_r1098019971
##########
server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java:
##########
@@ -353,11 +363,63 @@ 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:
+ *
+ * <pre>
+ * instance.volume.config.<volume-uri>.<hdfs-property>
+ * </pre>
+ *
+ * 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<String,String> customProps =
+
conf.getAllPropertiesWithPrefixStripped(Property.INSTANCE_VOLUMES_CONFIG);
+ customProps.forEach((key, value) -> {
+ if (key.startsWith(filesystemURI)) {
+ String property = key.substring(filesystemURI.length() + 1);
+ log.info("Overriding property {} to {} for volume {}", property,
value, filesystemURI);
+ volumeConfig.set(property, value);
+ }
+ });
+ return volumeConfig;
+ }
+
+ private static void warnVolumeOverridesMissingVolume(AccumuloConfiguration
conf,
+ Set<String> definedVolumes) {
+ final Map<String,String> overrideProperties = new ConcurrentHashMap<>(
+
conf.getAllPropertiesWithPrefixStripped(Property.INSTANCE_VOLUMES_CONFIG));
+
+ definedVolumes.forEach(vol -> {
+ log.debug("Looking for defined volume: {}", vol);
+ overrideProperties.keySet().forEach(override -> {
+ if (override.startsWith(vol)) {
+ log.debug("Found volume {}, removing property {}", vol, override);
+ overrideProperties.remove(override);
+ }
+ });
+ });
+
+ overrideProperties.forEach((k, v) -> log
+ .warn("Found no matching volume for volume config override property {}
= {}", k, v));
+ }
Review Comment:
On second thought, the entire method can be written without an intermediate
map or dealing with any concurrency... just grab the properties you want to log
a warning about.
```suggestion
private static void warnVolumeOverridesMissingVolume(AccumuloConfiguration
conf,
Set<String> definedVolumes) {
conf.getAllPropertiesWithPrefixStripped(Property.INSTANCE_VOLUMES_CONFIG).entrySet().stream()
// log only configs where none of the volumes (with a dot) prefix
its key
.filter(e -> definedVolumes.stream().noneMatch(vol ->
e.getKey().startsWith(vol + ".")))
.forEach(
e -> log.warn("Found no matching volume for volume config
override property {}", e));
}
```
##########
core/src/main/java/org/apache/accumulo/core/conf/Property.java:
##########
@@ -120,6 +120,16 @@ public enum Property {
+ " a comma or other reserved characters in a URI use standard URI
hex"
+ " encoding. For example replace commas with %2C.",
"1.6.0"),
+ INSTANCE_VOLUMES_CONFIG("instance.volumes.config.", null,
PropertyType.PREFIX,
+ "Properties in this category are used to provide volume specific
overrides to "
+ + "the general filesystem client configuration. Properties using
this prefix "
+ + "should be in the form "
+ +
"'instance.volumes.config.<volume-uri>.<property-name>=<property-value>. An "
+ + "example: "
+ +
"'instance.volume.config.hdfs://namespace-a:8020/accumulo.dfs.client.hedged.read.threadpool.size=10'.
"
+ + "Note that when specifying property names that contain colons in
the properties "
+ + "files that the colons need to be escaped with a backslash.",
+ "2.1.1"),
Review Comment:
Throughout this PR, you are inconsistently switching between
`instance.volumes.config` and `instance.volume.config`. Is this singular or
plural? Is it intended to apply to one volume only or more than one? I think
this needs to be cleaned up throughout.
##########
server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java:
##########
@@ -353,11 +363,63 @@ 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:
+ *
+ * <pre>
+ * instance.volume.config.<volume-uri>.<hdfs-property>
+ * </pre>
+ *
+ * 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<String,String> customProps =
+
conf.getAllPropertiesWithPrefixStripped(Property.INSTANCE_VOLUMES_CONFIG);
+ customProps.forEach((key, value) -> {
+ if (key.startsWith(filesystemURI)) {
+ String property = key.substring(filesystemURI.length() + 1);
+ log.info("Overriding property {} to {} for volume {}", property,
value, filesystemURI);
+ volumeConfig.set(property, value);
+ }
+ });
+ return volumeConfig;
+ }
+
+ private static void warnVolumeOverridesMissingVolume(AccumuloConfiguration
conf,
+ Set<String> definedVolumes) {
+ final Map<String,String> overrideProperties = new ConcurrentHashMap<>(
+
conf.getAllPropertiesWithPrefixStripped(Property.INSTANCE_VOLUMES_CONFIG));
+
+ definedVolumes.forEach(vol -> {
+ log.debug("Looking for defined volume: {}", vol);
+ overrideProperties.keySet().forEach(override -> {
+ if (override.startsWith(vol)) {
+ log.debug("Found volume {}, removing property {}", vol, override);
+ overrideProperties.remove(override);
+ }
+ });
+ });
Review Comment:
This can be written much more succinctly. Plus, you can get false positives
if you don't include the dot separator (for example, an override for
`hdfs://host:port/path/to/accumulo2` will incorrectly be removed as a match
against a volume named `hdfs://host:port/path/to/accumulo` if you don't include
the dot to ensure the volume name is terminated).
```suggestion
definedVolumes.forEach(
vol -> overrideProperties.entrySet().removeIf(entry ->
entry.getKey().startsWith(vol + ".")));
```
##########
minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java:
##########
@@ -47,6 +47,12 @@
* @since 1.6.0
*/
public class MiniAccumuloConfigImpl {
+
+ @FunctionalInterface
+ public static interface FinalSiteConfigUpdater {
+ void update(MiniAccumuloConfigImpl cfg);
+ }
Review Comment:
You don't need this interface. Just use `Consumer<MiniAccumuloConfigImpl>`.
While it probably doesn't hurt to have an extra interface to select a more
expressive name for ease of use, I think the name here, with the word "Final"
here is a bit presumptive (what if we need a subsequent modification to support
other tests? would we call it "FinalFinal"?). We can just avoid coming up with
a name by just using the existing Consumer... also, Consumer has the helpful
`andThen` method, which we don't need to reinvent and could be useful.
##########
minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java:
##########
@@ -812,4 +820,29 @@ public int getNumCompactors() {
public void setNumCompactors(int numCompactors) {
this.numCompactors = numCompactors;
}
+
+ /**
+ * Set the FinalSiteConfigUpdater instance that will be used to modify the
site configuration
+ * right before it's written out a file. This would be useful in the case
where the configuration
+ * needs to be updated based on a property that is set in
MiniAccumuloClusterImpl like
+ * instance.volumes
+ *
+ * @param updater FinalSiteConfigUpdater instance
+ * @since 2.1.1
+ */
+ public void setFinalSiteConfigUpdater(FinalSiteConfigUpdater updater) {
Review Comment:
Even if the parameter is changed to a Consumer, this method still needs a
decent name. I still have reservations about this one. One thing it has going
for it is that it's not public API... it's only on the Impl class.
Still, maybe something like:
```suggestion
public void preStartProcessor(Consumer<MiniAccumuloConfigImpl>
prestartConfigProcessor) {
```
##########
minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java:
##########
@@ -812,4 +820,29 @@ public int getNumCompactors() {
public void setNumCompactors(int numCompactors) {
this.numCompactors = numCompactors;
}
+
+ /**
+ * Set the FinalSiteConfigUpdater instance that will be used to modify the
site configuration
+ * right before it's written out a file. This would be useful in the case
where the configuration
+ * needs to be updated based on a property that is set in
MiniAccumuloClusterImpl like
+ * instance.volumes
+ *
+ * @param updater FinalSiteConfigUpdater instance
+ * @since 2.1.1
+ */
+ public void setFinalSiteConfigUpdater(FinalSiteConfigUpdater updater) {
+ this.updater = updater;
+ }
+
+ /**
+ * Called by MiniAccumuloClusterImpl after all modifications are done to the
configuration and
+ * right before it's written out to a file.
+ *
+ * @since 2.1.1
Review Comment:
Don't need `@since` for these internal Impl methods. If we start doing that
everywhere that's not public API, it will get unwieldy fast.
##########
server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java:
##########
@@ -353,11 +363,63 @@ 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:
+ *
+ * <pre>
+ * instance.volume.config.<volume-uri>.<hdfs-property>
+ * </pre>
+ *
+ * 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<String,String> customProps =
+
conf.getAllPropertiesWithPrefixStripped(Property.INSTANCE_VOLUMES_CONFIG);
+ customProps.forEach((key, value) -> {
+ if (key.startsWith(filesystemURI)) {
+ String property = key.substring(filesystemURI.length() + 1);
+ log.info("Overriding property {} to {} for volume {}", property,
value, filesystemURI);
+ volumeConfig.set(property, value);
+ }
+ });
Review Comment:
This can be written slightly more succinctly without an intermediate map
variable. This doesn't save much, but I do think there's some expressive value
in separating the Predicate `if` clause from the `forEach` action:
```suggestion
conf.getAllPropertiesWithPrefixStripped(Property.INSTANCE_VOLUMES_CONFIG).entrySet().stream()
.filter(e -> e.getKey().startsWith(filesystemURI + ".")).forEach(e
-> {
String key = e.getKey().substring(filesystemURI.length() + 1);
String value = e.getValue();
log.info("Overriding property {} for volume {}", key, value,
filesystemURI);
volumeConfig.set(key, value);
});
```
And again, you need to include the dot when checking for the startsWith...
or you'll get false positives from similarly named paths.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]