Github user ivakegg commented on a diff in the pull request:
https://github.com/apache/accumulo/pull/253#discussion_r129962445
--- Diff:
server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
---
@@ -18,61 +18,146 @@
import java.util.concurrent.ConcurrentHashMap;
-import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.server.client.HdfsZooInstance;
import org.apache.accumulo.server.conf.ServerConfigurationFactory;
import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
- * A {@link VolumeChooser} that delegates to another volume chooser based
on the presence of an experimental table property,
- * {@link Property#TABLE_VOLUME_CHOOSER}. If it isn't found, defaults back
to {@link RandomVolumeChooser}.
+ * A {@link VolumeChooser} that delegates to another volume chooser based
on other properties: table.custom.volume.chooser for tables, and
+ * general.custom.scoped.volume.chooser for scopes.
general.custor.{scope}.volume.chooser can override the system wide setting for
+ * general.custom.scoped.volume.chooser. At the this this was written, the
only known scope was "logger".
*/
public class PerTableVolumeChooser implements VolumeChooser {
-
- private final VolumeChooser fallbackVolumeChooser = new
RandomVolumeChooser();
+ private static final Logger log =
LoggerFactory.getLogger(PerTableVolumeChooser.class);
// TODO Add hint of expected size to construction, see ACCUMULO-3410
/* Track VolumeChooser instances so they can keep state. */
- private final ConcurrentHashMap<Table.ID,VolumeChooser>
tableSpecificChooser = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap<String,VolumeChooser>
tableSpecificChooser = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap<String,VolumeChooser>
scopeSpecificChooser = new ConcurrentHashMap<>();
+ private final RandomVolumeChooser randomChooser = new
RandomVolumeChooser();
+
// TODO has to be lazily initialized currently because of the reliance
on HdfsZooInstance. see ACCUMULO-3411
private volatile ServerConfigurationFactory serverConfs;
+ public static final String TABLE_VOLUME_CHOOSER =
Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "volume.chooser";
+
+ public static final String SCOPED_VOLUME_CHOOSER(String scope) {
+ return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope +
".volume.chooser";
+ }
+
+ public static final String DEFAULT_SCOPED_VOLUME_CHOOSER =
SCOPED_VOLUME_CHOOSER("scoped");
+
@Override
- public String choose(VolumeChooserEnvironment env, String[] options) {
- VolumeChooser chooser = null;
- if (env.hasTableId()) {
- // This local variable is an intentional component of the
single-check idiom.
- ServerConfigurationFactory localConf = serverConfs;
- if (localConf == null) {
- // If we're under contention when first getting here we'll throw
away some initializations.
- localConf = new
ServerConfigurationFactory(HdfsZooInstance.getInstance());
- serverConfs = localConf;
+ public String choose(VolumeChooserEnvironment env, String[] options)
throws AccumuloException {
+ if (log.isTraceEnabled()) {
+ log.trace("PerTableVolumeChooser.choose");
+ }
+ VolumeChooser chooser;
+ if (!env.hasTableId() && !env.hasScope()) {
+ // Should only get here during Initialize. Configurations are not
yet available.
+ return randomChooser.choose(env, options);
+ }
+
+ ServerConfigurationFactory localConf = loadConf();
+ if (env.hasScope()) {
+ // use the system configuration
+ chooser = getVolumeChooserForNonTable(env, localConf);
+ } else { // if (env.hasTableId()) {
+ // use the table configuration
+ chooser = getVolumeChooserForTable(env, localConf);
+ }
+
+ return chooser.choose(env, options);
+ }
+
+ private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment
env, ServerConfigurationFactory localConf) throws AccumuloException {
+ if (log.isTraceEnabled()) {
+ log.trace("Looking up property " + TABLE_VOLUME_CHOOSER + " for
Table id: " + env.getTableId());
+ }
+ final TableConfiguration tableConf =
localConf.getTableConfiguration(env.getTableId());
+ String clazz = tableConf.get(TABLE_VOLUME_CHOOSER);
+
+ return createVolumeChooser(clazz, TABLE_VOLUME_CHOOSER,
env.getTableId().canonicalID(), tableSpecificChooser);
+ }
+
+ private VolumeChooser
getVolumeChooserForNonTable(VolumeChooserEnvironment env,
ServerConfigurationFactory localConf) throws AccumuloException {
+ String property = SCOPED_VOLUME_CHOOSER(env.getScope());
+
+ if (log.isTraceEnabled()) {
+ log.trace("Looking up property: " + property);
+ }
+
+ AccumuloConfiguration systemConfiguration =
localConf.getSystemConfiguration();
+ String clazz = systemConfiguration.get(property);
+ // only if the custom property is not set to we fallback to the table
volume chooser setting
+ if (null == clazz) {
+ log.debug("Property not found: " + property + " using " +
DEFAULT_SCOPED_VOLUME_CHOOSER);
+ property = DEFAULT_SCOPED_VOLUME_CHOOSER;
--- End diff --
That was the intended design. But I suppose the message should spell out
both possibilities. I will work that.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---