You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@accumulo.apache.org by ctubbsii <gi...@git.apache.org> on 2017/04/21 20:02:33 UTC

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

GitHub user ctubbsii opened a pull request:

    https://github.com/apache/accumulo/pull/253

    ACCUMULO-4086 Improve volume chooser fallback

    Implement a better fallback mechanism when the user's selected
    VolumeChooser implementation cannot be loaded or is not specified.
    Handles all such cases, including non-table scopes (logger), and
    per-table scopes.
    
    ****
    
    Note: this isn't quite ready for merging, but since I'm trying to proxy code written by somebody else, I think it'd be helpful to have it inspected/reviewed by more eyes. I tried to run the tests, but they hung (in Eclipse, at least). Keep in mind that this has been rebase'd from an older version of Accumulo, and probably needs a bit more work.
    
    The goal of this is described in [ACCUMULO-4086][1] and its parent task, but all I have is the implementation, and it's not entirely clear what the best design is, in order to make sure this particular implementation matches that design.
    
    Any and all feedback/assistance is helpful. However, I'm already aware documentation is significantly lacking.... will address that once the design and implementation is hammered out... and that's a bit fuzzy to me right now. For example, I see that the patch modifies the default configs, and it's not clear to me what design goal that achieves.
    
    [1]: https://issues.apache.org/jira/browse/ACCUMULO-4086

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ctubbsii/accumulo ACCUMULO-4086-volume-chooser-fallback

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/accumulo/pull/253.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #253
    
----
commit 1c81d6e44a4127ae74852bc8246396878d05cef6
Author: Christopher Tubbs <ct...@apache.org>
Date:   2017-04-21T19:41:57Z

    ACCUMULO-4086 Improve volume chooser fallback
    
    Implement a better fallback mechanism when the user's selected
    VolumeChooser implementation cannot be loaded or is not specified.
    Handles all such cases, including non-table scopes (logger), and
    per-table scopes.

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r130183948
  
    --- 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);
    --- End diff --
    
    OK, I changed it back to return the random chooser on empty scope and non-table (or init scope).  I think I am done with changes.  Please merge in my pull request and the go ahead and refactor for a enumerated scope.  Remember that you cannot load the configuration when the incoming scope is INIT as the zookeeper initialization will not have been done yet.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129964682
  
    --- 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;
    +      clazz = systemConfiguration.get(DEFAULT_SCOPED_VOLUME_CHOOSER);
    +    }
    +
    +    return createVolumeChooser(clazz, property, env.getScope(), scopeSpecificChooser);
    +  }
    +
    +  private VolumeChooser createVolumeChooser(String clazz, String property, String key, ConcurrentHashMap<String,VolumeChooser> cache) throws AccumuloException {
    +    if (null == clazz || clazz.isEmpty()) {
    +      String msg = "Property " + property + " must be set" + (null == clazz ? " " : " properly ") + "to use the " + getClass().getSimpleName();
    +      log.error(msg);
    +      throw new AccumuloException(msg);
    +    }
    +
    +    VolumeChooser chooser = cache.get(key);
    +    if (chooser == null || !(chooser.getClass().getName().equals(clazz))) {
    +      if (log.isTraceEnabled() && chooser != null) {
    +        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    +        log.trace("Change detected for " + property + " for " + key);
    +      }
    +      VolumeChooser temp;
    +      try {
    +        temp = loadClass(clazz);
    +      } catch (Exception e) {
    +        String msg = "Failed to create instance for " + key + " configured to use " + clazz + " via " + property;
    +        log.error(msg);
    +        throw new AccumuloException(msg, e);
           }
    -      final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -      chooser = tableSpecificChooser.get(env.getTableId());
           if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    +        chooser = cache.putIfAbsent(key, temp);
    --- End diff --
    
    not true.  If our call to putIfAbsent wins, then null is returned meaning that temp is the one to use.  Otherwise we should use the one in the map which would have been returned.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113243125
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -18,36 +18,65 @@
     
     import java.util.concurrent.ConcurrentHashMap;
     
    +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}.
      */
     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<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;
    +  private volatile VolumeChooser fallbackVolumeChooser = null;
     
       @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;
    -      }
    -      final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    +  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();
    +    lazilyCreateFallbackChooser();
    +    if (env.hasScope()) {
    --- End diff --
    
    Perhaps ignorance on my part, but shouldn't env.hasTableId() be checked first?  Can env.hasTableId() && env.hasScope()?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129963947
  
    --- 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<>();
    --- End diff --
    
    No, that's fine. I just wanted to make sure it wasn't a bad merge of the Table.ID work that @milleruntime did.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129887994
  
    --- Diff: assemble/conf/accumulo-site.xml ---
    @@ -39,6 +39,15 @@
         <name>tserver.memory.maps.native.enabled</name>
         <value>true</value>
       </property>
    +  <!-- Used by the PerTableVolumeChooser which is the default general.volume.chooser -->
    --- End diff --
    
    I agree. I think we'd want things to work out of the box. I think this is why we previously discussed changing the default back to random instead of per-table.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129968372
  
    --- 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;
    +      clazz = systemConfiguration.get(DEFAULT_SCOPED_VOLUME_CHOOSER);
    +    }
    +
    +    return createVolumeChooser(clazz, property, env.getScope(), scopeSpecificChooser);
    +  }
    +
    +  private VolumeChooser createVolumeChooser(String clazz, String property, String key, ConcurrentHashMap<String,VolumeChooser> cache) throws AccumuloException {
    +    if (null == clazz || clazz.isEmpty()) {
    +      String msg = "Property " + property + " must be set" + (null == clazz ? " " : " properly ") + "to use the " + getClass().getSimpleName();
    +      log.error(msg);
    +      throw new AccumuloException(msg);
    +    }
    +
    +    VolumeChooser chooser = cache.get(key);
    +    if (chooser == null || !(chooser.getClass().getName().equals(clazz))) {
    +      if (log.isTraceEnabled() && chooser != null) {
    +        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    +        log.trace("Change detected for " + property + " for " + key);
    +      }
    +      VolumeChooser temp;
    +      try {
    +        temp = loadClass(clazz);
    +      } catch (Exception e) {
    +        String msg = "Failed to create instance for " + key + " configured to use " + clazz + " via " + property;
    +        log.error(msg);
    +        throw new AccumuloException(msg, e);
           }
    -      final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -      chooser = tableSpecificChooser.get(env.getTableId());
           if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    +        chooser = cache.putIfAbsent(key, temp);
    --- End diff --
    
    Ah, I was thinking putIfAbsent worked like computeIfAbsent.
    This code is simplified if we just use:
    ```java
    chooser = cache.computeIfAbsent(key, k -> temp);
    ```
    
    Some of the surrounding code is a bit complicated. I think there's probably some room for cleanup or comments to explain the intended behavior it is trying to achieve with the cache, and monitoring for an updated chooser, etc.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129873342
  
    --- Diff: assemble/conf/accumulo-site.xml ---
    @@ -39,6 +39,15 @@
         <name>tserver.memory.maps.native.enabled</name>
         <value>true</value>
       </property>
    +  <!-- Used by the PerTableVolumeChooser which is the default general.volume.chooser -->
    --- End diff --
    
    The big question that remains is since the PerTableVolumeChooser is the default, then the system will not work until the table and scoped volume choosers are set.  Hence I added them here in the example accumulo-site.xml and configured them in the mini accumulo configuration.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129888546
  
    --- 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<>();
    --- End diff --
    
    I think these are supposed to be Table.ID


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    @ctubbsii  I am attempting to resolve the issues within this pull request.  I am creating a separate branch under my account.  Shall I subsequently create a pull request from my branch to yours so that you can update this pull request or is there another way to do this?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113242911
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -21,84 +21,90 @@
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collections;
    -import java.util.HashMap;
     import java.util.HashSet;
     import java.util.Map;
     import java.util.Set;
    -import java.util.function.Predicate;
     
    +import org.apache.accumulo.core.client.AccumuloException;
     import org.apache.accumulo.core.conf.Property;
     import org.apache.accumulo.core.volume.Volume;
     import org.apache.accumulo.server.client.HdfsZooInstance;
     import org.apache.accumulo.server.conf.ServerConfigurationFactory;
    -import org.apache.accumulo.server.conf.TableConfiguration;
     import org.apache.commons.collections.map.LRUMap;
     import org.apache.commons.lang.StringUtils;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX + "preferredVolumes";
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
           return super.choose(env, options);
    +    }
    +    ServerConfigurationFactory localConf = loadConf();
     
    -    // Get the current table's properties, and find the preferred volumes property
    -    // 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;
    +    String systemPreferredVolumes = localConf.getConfiguration().get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    if (null == systemPreferredVolumes || systemPreferredVolumes.isEmpty()) {
    +      String logMessage = "Default preferred volumes are missing.";
    +      log.debug(logMessage);
    +      throw new AccumuloException(logMessage);
         }
    -    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -    final Map<String,String> props = new HashMap<>();
    -    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
    -    if (props.isEmpty()) {
    -      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
    -      return super.choose(env, options);
    +
    +    String volumes = null;
    +    if (env.hasTableId()) {
    +      volumes = localConf.getTableConfiguration(env.getTableId()).get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    } else if (env.hasScope()) {
    +      volumes = localConf.getConfiguration().get(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".preferredVolumes");
    --- End diff --
    
    Would it make sense to add static that takes a scope argument and generated this property?  This could be a public static equivalent to `PREFERRED_VOLUMES_CUSTOM_KEY` for configuration purposes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113243918
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -68,10 +100,107 @@ public String choose(VolumeChooserEnvironment env, String[] options) {
               chooser = last;
             }
           }
    +    }
    +    return chooser;
    +  }
    +
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +    VolumeChooser chooser;
    +    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +
    +    if (log.isTraceEnabled()) {
    +      log.trace("Scope: " + env.getScope());
    +      log.trace("Looking up property: " + customProperty);
    +    }
    +
    +    AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    +    String clazz = systemConfiguration.get(customProperty);
    +    if (null == clazz || clazz.isEmpty()) {
    +      log.debug("Property not found: " + customProperty + " using fallback chooser.");
    +      return fallbackVolumeChooser;
         } else {
    -      chooser = fallbackVolumeChooser;
    +      chooser = scopeSpecificChooser.get(env.getScope());
    +      if (chooser == null) {
    +        VolumeChooser temp;
    +        try {
    +          temp = loadClassForCustomProperty(clazz);
    +        } catch (Exception e) {
    +          log.error("Failed to create instance for " + env.getScope() + " configured to use " + clazz + " via " + customProperty);
    +          return fallbackVolumeChooser;
    +        }
    +        chooser = scopeSpecificChooser.putIfAbsent(env.getScope(), temp);
    +        if (chooser == null) {
    +          chooser = temp;
    +          // Otherwise, someone else beat us to initializing; use theirs.
    +        }
    +      } else if (!(chooser.getClass().getName().equals(clazz))) {
    +        if (log.isTraceEnabled()) {
    +          log.trace("change detected for scope: " + env.getScope());
    +        }
    +        // the configuration for this scope's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
    +        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    +        VolumeChooser temp;
    +        try {
    +          temp = loadClassForCustomProperty(clazz);
    +        } catch (Exception e) {
    +          log.error("Failed to create instance for " + env.getScope() + " configured to use " + clazz + " via " + customProperty);
    +          return fallbackVolumeChooser;
    --- End diff --
    
    Should we not be failing here instead of using the fallback?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113297684
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -68,10 +100,107 @@ public String choose(VolumeChooserEnvironment env, String[] options) {
               chooser = last;
             }
           }
    +    }
    +    return chooser;
    +  }
    +
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +    VolumeChooser chooser;
    +    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +
    +    if (log.isTraceEnabled()) {
    +      log.trace("Scope: " + env.getScope());
    +      log.trace("Looking up property: " + customProperty);
    +    }
    +
    +    AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    +    String clazz = systemConfiguration.get(customProperty);
    +    if (null == clazz || clazz.isEmpty()) {
    +      log.debug("Property not found: " + customProperty + " using fallback chooser.");
    +      return fallbackVolumeChooser;
         } else {
    -      chooser = fallbackVolumeChooser;
    +      chooser = scopeSpecificChooser.get(env.getScope());
    +      if (chooser == null) {
    +        VolumeChooser temp;
    +        try {
    +          temp = loadClassForCustomProperty(clazz);
    +        } catch (Exception e) {
    +          log.error("Failed to create instance for " + env.getScope() + " configured to use " + clazz + " via " + customProperty);
    +          return fallbackVolumeChooser;
    +        }
    +        chooser = scopeSpecificChooser.putIfAbsent(env.getScope(), temp);
    +        if (chooser == null) {
    +          chooser = temp;
    +          // Otherwise, someone else beat us to initializing; use theirs.
    +        }
    +      } else if (!(chooser.getClass().getName().equals(clazz))) {
    +        if (log.isTraceEnabled()) {
    +          log.trace("change detected for scope: " + env.getScope());
    +        }
    +        // the configuration for this scope's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
    +        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    +        VolumeChooser temp;
    +        try {
    +          temp = loadClassForCustomProperty(clazz);
    +        } catch (Exception e) {
    +          log.error("Failed to create instance for " + env.getScope() + " configured to use " + clazz + " via " + customProperty);
    +          return fallbackVolumeChooser;
    --- End diff --
    
    I think we should fail-hard here instead of the current behavior. Matt's original code was to fall back, but we can simplify this, because I don't think that's necessary.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129889643
  
    --- 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()) {
    --- End diff --
    
    Don't think this protective log enable check is necessary with slf4j.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113294271
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -21,84 +21,90 @@
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collections;
    -import java.util.HashMap;
     import java.util.HashSet;
     import java.util.Map;
     import java.util.Set;
    -import java.util.function.Predicate;
     
    +import org.apache.accumulo.core.client.AccumuloException;
     import org.apache.accumulo.core.conf.Property;
     import org.apache.accumulo.core.volume.Volume;
     import org.apache.accumulo.server.client.HdfsZooInstance;
     import org.apache.accumulo.server.conf.ServerConfigurationFactory;
    -import org.apache.accumulo.server.conf.TableConfiguration;
     import org.apache.commons.collections.map.LRUMap;
     import org.apache.commons.lang.StringUtils;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX + "preferredVolumes";
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
           return super.choose(env, options);
    +    }
    +    ServerConfigurationFactory localConf = loadConf();
     
    -    // Get the current table's properties, and find the preferred volumes property
    -    // 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;
    +    String systemPreferredVolumes = localConf.getConfiguration().get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    if (null == systemPreferredVolumes || systemPreferredVolumes.isEmpty()) {
    +      String logMessage = "Default preferred volumes are missing.";
    +      log.debug(logMessage);
    +      throw new AccumuloException(logMessage);
         }
    -    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -    final Map<String,String> props = new HashMap<>();
    -    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
    -    if (props.isEmpty()) {
    -      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
    -      return super.choose(env, options);
    +
    +    String volumes = null;
    +    if (env.hasTableId()) {
    +      volumes = localConf.getTableConfiguration(env.getTableId()).get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    } else if (env.hasScope()) {
    +      volumes = localConf.getConfiguration().get(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".preferredVolumes");
    --- End diff --
    
    Yeah, that'd be a nice improvement.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r130121958
  
    --- 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);
    --- End diff --
    
    ok, this decision is the reason the VolumeChooserIT fails.  This requires us to access the configuration when the table id and the env are empty.  This happens only once when accumulo is being initialized.  The code is as follows:
    
        final String rootTabletDir = new Path(fs.choose(chooserEnv, configuredVolumes) + Path.SEPARATOR + ServerConstants.TABLE_DIR + Path.SEPARATOR + RootTable.ID
            + RootTable.ROOT_TABLET_LOCATION).toString();
    
        try {
          initZooKeeper(opts, uuid.toString(), instanceNamePath, rootTabletDir);
        } catch (Exception e) {
          log.error("FATAL: Failed to initialize zookeeper", e);
          return false;
        }
    
    You can see that we are getting a volume for the root tablet before we initialize zookeeper.  Hence at that time the configuration is actually not initialized.  The failure occurs with this stack trace:
    
    2017-07-28 11:12:48,995 [init.Initialize] ERROR: Fatal exception
    java.lang.RuntimeException: Accumulo not initialized, there is no instance id at file:/local/projects/ivakegg-accumulo.git/test/target/mini-tests/org.apache.accumulo.test.VolumeChooserIT_twoTablesRandomVolumeChooser/volumes/v1/instance_id
            at org.apache.accumulo.core.zookeeper.ZooUtil.getInstanceIDFromHdfs(ZooUtil.java:66)
            at org.apache.accumulo.core.zookeeper.ZooUtil.getInstanceIDFromHdfs(ZooUtil.java:51)
            at org.apache.accumulo.server.client.HdfsZooInstance._getInstanceID(HdfsZooInstance.java:154)
            at org.apache.accumulo.server.client.HdfsZooInstance.getInstanceID(HdfsZooInstance.java:138)
            at org.apache.accumulo.server.conf.ServerConfigurationFactory.<init>(ServerConfigurationFactory.java:93)
            at org.apache.accumulo.server.fs.PerTableVolumeChooser.loadConf(PerTableVolumeChooser.java:162)
            at org.apache.accumulo.server.fs.PerTableVolumeChooser.choose(PerTableVolumeChooser.java:60)
            at org.apache.accumulo.server.fs.VolumeManagerImpl.choose(VolumeManagerImpl.java:480)
            at org.apache.accumulo.server.init.Initialize.initialize(Initialize.java:328)
            at org.apache.accumulo.server.init.Initialize.doInit(Initialize.java:319)
            at org.apache.accumulo.server.init.Initialize.execute(Initialize.java:810)
            at org.apache.accumulo.server.init.Initialize.main(Initialize.java:819)
            at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
            at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
            at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
            at java.lang.reflect.Method.invoke(Method.java:498)
            at org.apache.accumulo.start.Main$2.run(Main.java:161)
            at java.lang.Thread.run(Thread.java:745)
    
    So in summary I believe we need to return the random chooser in this circumstance.  Savvy?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129914902
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -39,73 +40,139 @@
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String TABLE_PREFERRED_VOLUMES = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "preferred.volumes";
    +
    +  public static final String SCOPED_PREFERRED_VOLUMES(String scope) {
    +    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope + ".preferred.volumes";
    +  }
    +
    +  public static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = SCOPED_PREFERRED_VOLUMES("scoped");
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
           return super.choose(env, options);
    +    }
     
    -    // Get the current table's properties, and find the preferred volumes property
    -    // 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;
    +    // get the volumes property
    +    ServerConfigurationFactory localConf = loadConf();
    +    List<String> volumes;
    +    if (env.hasTableId()) {
    +      volumes = getPreferredVolumesForTable(env, localConf, options);
    +      // localConf.getTableConfiguration(env.getTableId()).get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    } else { // env.hasScope()
    +      volumes = getPreferredVolumesForNonTable(env, localConf, options);
    +      // localConf.getSystemConfiguration().get(PREFERRED_VOLUMES_SCOPED_KEY(env.getScope()));
         }
    -    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -    final Map<String,String> props = new HashMap<>();
    -    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
    -    if (props.isEmpty()) {
    -      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
    -      return super.choose(env, options);
    +
    +    // Randomly choose the volume from the preferred volumes
    +    String choice = super.choose(env, volumes.toArray(EMPTY_STRING_ARRAY));
    +    if (log.isTraceEnabled()) {
    +      log.trace("Choice = " + choice);
         }
    -    String volumes = props.get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    return choice;
    +  }
     
    +  private List<String> getPreferredVolumesForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf, String[] options)
    +      throws AccumuloException {
         if (log.isTraceEnabled()) {
    -      log.trace("In custom chooser");
    -      log.trace("Volumes: " + volumes);
    -      log.trace("TableID: " + env.getTableId());
    +      log.trace("Looking up property " + TABLE_PREFERRED_VOLUMES + " for Table id: " + env.getTableId());
         }
    -    // If the preferred volumes property was specified, split the returned string by the comma and add use it to filter the given options.
    -    Set<String> preferred = parsedPreferredVolumes.get(volumes);
    -    if (preferred == null) {
    -      preferred = new HashSet<>(Arrays.asList(StringUtils.split(volumes, ',')));
    -      parsedPreferredVolumes.put(volumes, preferred);
    +    final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    +    String volumes = tableConf.get(TABLE_PREFERRED_VOLUMES);
    +
    +    return getFilteredOptions(TABLE_PREFERRED_VOLUMES, volumes, options);
    +  }
    +
    +  private List<String> getPreferredVolumesForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf, String[] options)
    +      throws AccumuloException {
    +    String property = SCOPED_PREFERRED_VOLUMES(env.getScope());
    +
    +    if (log.isTraceEnabled()) {
    +      log.trace("Looking up property: " + property);
    +    }
    +
    +    AccumuloConfiguration systemConfiguration = localConf.getSystemConfiguration();
    +    String volumes = systemConfiguration.get(property);
    +
    +    // only if the custom property is not set to we fallback to the default scoped preferred volumes
    +    if (null == volumes) {
    +      log.debug("Property not found: " + property + " using " + DEFAULT_SCOPED_PREFERRED_VOLUMES);
    +      property = DEFAULT_SCOPED_PREFERRED_VOLUMES;
    +      volumes = systemConfiguration.get(DEFAULT_SCOPED_PREFERRED_VOLUMES);
    --- End diff --
    
    Again, this one is handled differently, and will not fail as expected, if a scope's chooser is not set. And, falling back to a catch-all chooser for all scopes I'm not sure is a good idea.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129917588
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java ---
    @@ -464,16 +472,22 @@ public ContentSummary getContentSummary(Path dir) throws IOException {
         return getVolumeByPath(dir).getFileSystem().getContentSummary(dir);
       }
     
    -  // Only used as a fall back if the configured chooser misbehaves.
    -  private final VolumeChooser failsafeChooser = new RandomVolumeChooser();
    -
       @Override
       public String choose(VolumeChooserEnvironment env, String[] options) {
    -    final String choice = chooser.choose(env, options);
    -    if (!(ArrayUtils.contains(options, choice))) {
    -      log.error("The configured volume chooser, '" + chooser.getClass() + "', or one of its delegates returned a volume not in the set of options provided; "
    -          + "will continue by relying on a RandomVolumeChooser. You should investigate and correct the named chooser.");
    -      return failsafeChooser.choose(env, options);
    +    final String choice;
    +    try {
    +      choice = chooser.choose(env, options);
    +
    +      if (!(ArrayUtils.contains(options, choice))) {
    +        // we may want to go with random if they chooser was not overridden
    +        String msg = "The configured volume chooser, '" + chooser.getClass()
    +            + "', or one of its delegates returned a volume not in the set of options provided";
    +        log.error(msg);
    --- End diff --
    
    Should probably not log and throw. We should log or throw, and if we throw, leave logging to the caller. This avoids spammy redundant logs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r119182922
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -67,86 +69,67 @@ public String choose(VolumeChooserEnvironment env, String[] options) throws Accu
         return chooser.choose(env, options);
       }
     
    -  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
         if (log.isTraceEnabled()) {
           log.trace("Table id: " + env.getTableId());
         }
         final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
         String clazz = tableConf.get(Property.TABLE_VOLUME_CHOOSER);
    -    if (null == clazz || clazz.isEmpty()) {
    -      chooser = fallbackVolumeChooser;
    -    } else {
    -      chooser = tableSpecificChooser.get(env.getTableId());
    -      if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    -        if (chooser == null) {
    -          chooser = temp;
    -          // Otherwise, someone else beat us to initializing; use theirs.
    -        }
    -      } else if (!(chooser.getClass().getName().equals(clazz))) {
    -        if (log.isTraceEnabled()) {
    -          log.trace("change detected for table id: " + env.getTableId());
    -        }
    -        // the configuration for this table's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
    -        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        VolumeChooser last = tableSpecificChooser.replace(env.getTableId(), temp);
    -        if (chooser.equals(last)) {
    -          chooser = temp;
    -        } else {
    -          // Someone else beat us to updating; use theirs.
    -          chooser = last;
    -        }
    -      }
    -    }
    -    return chooser;
    +
    +    return createVolumeChooser(clazz, Property.TABLE_VOLUME_CHOOSER.getKey(), env.getTableId(), tableSpecificChooser);
       }
     
    -  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
    -    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +    String property = VOLUME_CHOOSER_SCOPED_KEY(env.getScope());
     
         if (log.isTraceEnabled()) {
           log.trace("Scope: " + env.getScope());
    -      log.trace("Looking up property: " + customProperty);
    +      log.trace("Looking up property: " + property);
         }
     
         AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    -    String clazz = systemConfiguration.get(customProperty);
    +    String clazz = systemConfiguration.get(property);
    +    // only if the custom property is not set to we fallback to the table volumn chooser setting
    +    if (null == clazz) {
    --- End diff --
    
    @matthpeterson  I'm not sure I follow. The implementation logic for `PerTableVolumeChooser` is basically:
    
    ```
    if (per-table-scope) {
      delegate = getTableConfig(table).getTableChooser()
    } else { // non-table scope
      delegate = getSystemConfiguration().getScopeChooser(scope)
    }
    delegate.choose()
    ```
    
    If resolving the delegate chooser fails, we should simply report the error. Is that not what we're doing? What have I overlooked?
    
    As for the property name, etc. ... I don't think we should add new baked-in properties which possibly only apply to the implementation of one specific chooser. I'm still not sold 100% on the `table.volume.chooser` property being baked-in rather than a custom property(1). It was modeled after the per-table balancer, which did that, so there was some precedent. At least it is still marked "experimental", so we have some flexibility to change things if we can agree on something better that is consistent with all the implementation configs (per-table and per-non-table scopes).
    
    (1): because it only applies to choosers who not only behave differently on different tables, but specifically those who do so by loading a different delegate chooser from per-table configs


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129960558
  
    --- 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<>();
    --- End diff --
    
    You will see that the same routine is used to add volume choosers into those maps, so I had to use a common denominator for the class.  String works well.  However if you insist that I use Table.ID here instead, then I will denormalize the common method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    I did another PR against this branch to make this compatible with the master.  Your change to the configuration factory object methods require changes to this PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129890129
  
    --- 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);
    --- End diff --
    
    This is a weird case for the per-table chooser to be forced to handle. It seems "init" should be a scope instead. Random should never just be used.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113297767
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -68,10 +100,107 @@ public String choose(VolumeChooserEnvironment env, String[] options) {
               chooser = last;
             }
           }
    +    }
    +    return chooser;
    +  }
    +
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +    VolumeChooser chooser;
    +    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +
    +    if (log.isTraceEnabled()) {
    +      log.trace("Scope: " + env.getScope());
    +      log.trace("Looking up property: " + customProperty);
    +    }
    +
    +    AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    +    String clazz = systemConfiguration.get(customProperty);
    +    if (null == clazz || clazz.isEmpty()) {
    +      log.debug("Property not found: " + customProperty + " using fallback chooser.");
    +      return fallbackVolumeChooser;
         } else {
    -      chooser = fallbackVolumeChooser;
    +      chooser = scopeSpecificChooser.get(env.getScope());
    +      if (chooser == null) {
    +        VolumeChooser temp;
    +        try {
    +          temp = loadClassForCustomProperty(clazz);
    +        } catch (Exception e) {
    +          log.error("Failed to create instance for " + env.getScope() + " configured to use " + clazz + " via " + customProperty);
    +          return fallbackVolumeChooser;
    --- End diff --
    
    Same comment as above. We can simplify this if we fail-hard.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129968227
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java ---
    @@ -464,16 +472,22 @@ public ContentSummary getContentSummary(Path dir) throws IOException {
         return getVolumeByPath(dir).getFileSystem().getContentSummary(dir);
       }
     
    -  // Only used as a fall back if the configured chooser misbehaves.
    -  private final VolumeChooser failsafeChooser = new RandomVolumeChooser();
    -
       @Override
       public String choose(VolumeChooserEnvironment env, String[] options) {
    -    final String choice = chooser.choose(env, options);
    -    if (!(ArrayUtils.contains(options, choice))) {
    -      log.error("The configured volume chooser, '" + chooser.getClass() + "', or one of its delegates returned a volume not in the set of options provided; "
    -          + "will continue by relying on a RandomVolumeChooser. You should investigate and correct the named chooser.");
    -      return failsafeChooser.choose(env, options);
    +    final String choice;
    +    try {
    +      choice = chooser.choose(env, options);
    +
    +      if (!(ArrayUtils.contains(options, choice))) {
    +        // we may want to go with random if they chooser was not overridden
    +        String msg = "The configured volume chooser, '" + chooser.getClass()
    +            + "', or one of its delegates returned a volume not in the set of options provided";
    +        log.error(msg);
    --- End diff --
    
    fair point....ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113294955
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -21,84 +21,90 @@
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collections;
    -import java.util.HashMap;
     import java.util.HashSet;
     import java.util.Map;
     import java.util.Set;
    -import java.util.function.Predicate;
     
    +import org.apache.accumulo.core.client.AccumuloException;
     import org.apache.accumulo.core.conf.Property;
     import org.apache.accumulo.core.volume.Volume;
     import org.apache.accumulo.server.client.HdfsZooInstance;
     import org.apache.accumulo.server.conf.ServerConfigurationFactory;
    -import org.apache.accumulo.server.conf.TableConfiguration;
     import org.apache.commons.collections.map.LRUMap;
     import org.apache.commons.lang.StringUtils;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX + "preferredVolumes";
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
           return super.choose(env, options);
    +    }
    +    ServerConfigurationFactory localConf = loadConf();
     
    -    // Get the current table's properties, and find the preferred volumes property
    -    // 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;
    +    String systemPreferredVolumes = localConf.getConfiguration().get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    if (null == systemPreferredVolumes || systemPreferredVolumes.isEmpty()) {
    +      String logMessage = "Default preferred volumes are missing.";
    +      log.debug(logMessage);
    +      throw new AccumuloException(logMessage);
         }
    -    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -    final Map<String,String> props = new HashMap<>();
    -    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
    -    if (props.isEmpty()) {
    -      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
    -      return super.choose(env, options);
    +
    +    String volumes = null;
    +    if (env.hasTableId()) {
    +      volumes = localConf.getTableConfiguration(env.getTableId()).get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    } else if (env.hasScope()) {
    +      volumes = localConf.getConfiguration().get(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".preferredVolumes");
    +    }
    +
    +    // if there was an empty or missing property, use the system-wide volumes
    +    if (null == volumes || volumes.isEmpty()) {
    +      if (env.hasTableId()) {
    +        log.warn("Missing property for TableID " + env.getTableId() + " but it should have picked up default volumes.");
    +      } else {
    +        log.debug("Missing preferred volumes for scope " + env.getScope() + ". Using default volumes.");
    --- End diff --
    
    This should probably be a hard fail also.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r119197637
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -67,86 +69,67 @@ public String choose(VolumeChooserEnvironment env, String[] options) throws Accu
         return chooser.choose(env, options);
       }
     
    -  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
         if (log.isTraceEnabled()) {
           log.trace("Table id: " + env.getTableId());
         }
         final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
         String clazz = tableConf.get(Property.TABLE_VOLUME_CHOOSER);
    -    if (null == clazz || clazz.isEmpty()) {
    -      chooser = fallbackVolumeChooser;
    -    } else {
    -      chooser = tableSpecificChooser.get(env.getTableId());
    -      if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    -        if (chooser == null) {
    -          chooser = temp;
    -          // Otherwise, someone else beat us to initializing; use theirs.
    -        }
    -      } else if (!(chooser.getClass().getName().equals(clazz))) {
    -        if (log.isTraceEnabled()) {
    -          log.trace("change detected for table id: " + env.getTableId());
    -        }
    -        // the configuration for this table's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
    -        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        VolumeChooser last = tableSpecificChooser.replace(env.getTableId(), temp);
    -        if (chooser.equals(last)) {
    -          chooser = temp;
    -        } else {
    -          // Someone else beat us to updating; use theirs.
    -          chooser = last;
    -        }
    -      }
    -    }
    -    return chooser;
    +
    +    return createVolumeChooser(clazz, Property.TABLE_VOLUME_CHOOSER.getKey(), env.getTableId(), tableSpecificChooser);
       }
     
    -  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
    -    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +    String property = VOLUME_CHOOSER_SCOPED_KEY(env.getScope());
     
         if (log.isTraceEnabled()) {
           log.trace("Scope: " + env.getScope());
    -      log.trace("Looking up property: " + customProperty);
    +      log.trace("Looking up property: " + property);
         }
     
         AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    -    String clazz = systemConfiguration.get(customProperty);
    +    String clazz = systemConfiguration.get(property);
    +    // only if the custom property is not set to we fallback to the table volumn chooser setting
    +    if (null == clazz) {
    --- End diff --
    
    @ivakegg Ah, yes, we should not be falling back to `table.volume.chooser` for the non-table scopes. It should simply fail. However, since non-table scopes are a new feature, it might cause things to fail if the non-table scope delegate hasn't been specified. With that in mind, I can see how it might be advantageous to consider a single property, rather than fail every time a new scope is added to the chooser environment.
    
    If we are sticking with the baked-in properties, perhaps: `general.volume.chooser.logger` and `general.volume.chooser.unknown`. More generally, `general.volume.chooser.<scope>`
    
    If we want to move to custom properties only, perhaps: `table.custom.volume.chooser` instead of `table.volume.chooser`. And, also the scopes at `general.custom.volume.chooser.<scope>`?
    
    We should also consider how other choosers get their table and non-table configs, rather than just the `PerTableVolumeChooser`. For example, the `PreferredVolumeChooser` I think gets its configs from `table.custom.preferredVolumes`, but I'm not sure that makes sense when handling non-table scopes.
    
    One alternative to the separate properties, which might make it easier to deal with the various scopes is that we could have more complex property types, whose values include the class name and its configuration serialized. I haven't thought too hard about what that would look like, but maybe something structured, like JSON.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    I concur that failing if a configured class fails to load is better then falling back to something else.  If a user went through the trouble of configuring something and they did it incorrectly, then the system should never fall back to something else because that may have unexpected and perhaps bad consequences.  Instead the user should be forced to fix the configuration.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129895600
  
    --- 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
    --- End diff --
    
    This comment does not appear to reflect the current behavior. It doesn't fall back to the table volume chooser. It falls back to a system wide default for all scopes, but which still needs to be set. This is different from the way it handles missing configuration for the per-table case. On the other hand, it seems the only way to make this never fail is for this to be aware of all possible scopes. Perhaps the different scopes should be an enum so this can be detected better?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r130078545
  
    --- 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;
    +      clazz = systemConfiguration.get(DEFAULT_SCOPED_VOLUME_CHOOSER);
    +    }
    +
    +    return createVolumeChooser(clazz, property, env.getScope(), scopeSpecificChooser);
    +  }
    +
    +  private VolumeChooser createVolumeChooser(String clazz, String property, String key, ConcurrentHashMap<String,VolumeChooser> cache) throws AccumuloException {
    +    if (null == clazz || clazz.isEmpty()) {
    +      String msg = "Property " + property + " must be set" + (null == clazz ? " " : " properly ") + "to use the " + getClass().getSimpleName();
    +      log.error(msg);
    +      throw new AccumuloException(msg);
    +    }
    +
    +    VolumeChooser chooser = cache.get(key);
    +    if (chooser == null || !(chooser.getClass().getName().equals(clazz))) {
    +      if (log.isTraceEnabled() && chooser != null) {
    +        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    +        log.trace("Change detected for " + property + " for " + key);
    +      }
    +      VolumeChooser temp;
    +      try {
    +        temp = loadClass(clazz);
    +      } catch (Exception e) {
    +        String msg = "Failed to create instance for " + key + " configured to use " + clazz + " via " + property;
    +        log.error(msg);
    +        throw new AccumuloException(msg, e);
           }
    -      final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -      chooser = tableSpecificChooser.get(env.getTableId());
           if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    +        chooser = cache.putIfAbsent(key, temp);
    --- End diff --
    
    OK, I am updating this section.  It appears the logic below this is not quite right anyway.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113297447
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/conf/Property.java ---
    @@ -191,7 +191,7 @@
           + "server-internal scheduled tasks"),
       // If you update the default type, be sure to update the default used for initialization failures in VolumeManagerImpl
       @Experimental
    -  GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.PerTableVolumeChooser", PropertyType.CLASSNAME,
    +  GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.RandomVolumeChooser", PropertyType.CLASSNAME,
    --- End diff --
    
    I'm actually questioning whether this is necessary myself. The original intent of the parent issue was to ban certain volumes from being used entirely (unless explicitly desired). This change was to help provide a sane fall-back scenario, but I'm not sure it's necessary if we change things to be a hard-fail in the case of misconfiguration.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129916561
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java ---
    @@ -28,5 +29,5 @@
      * benefit from using per-table configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
      */
     public interface VolumeChooser {
    -  String choose(VolumeChooserEnvironment env, String[] options);
    +  String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException;
    --- End diff --
    
    Can we avoid changing this interface?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129899084
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -39,73 +40,139 @@
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String TABLE_PREFERRED_VOLUMES = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "preferred.volumes";
    +
    +  public static final String SCOPED_PREFERRED_VOLUMES(String scope) {
    +    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope + ".preferred.volumes";
    +  }
    +
    +  public static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = SCOPED_PREFERRED_VOLUMES("scoped");
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
    --- End diff --
    
    Should this fail here instead?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129965171
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -39,73 +40,139 @@
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String TABLE_PREFERRED_VOLUMES = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "preferred.volumes";
    +
    +  public static final String SCOPED_PREFERRED_VOLUMES(String scope) {
    +    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope + ".preferred.volumes";
    +  }
    +
    +  public static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = SCOPED_PREFERRED_VOLUMES("scoped");
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
    --- End diff --
    
    As with the PerTableVolumeChooser, this case should probably default to the "init" scope.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    @ivakegg if you do a PR against my branch, I'll merge it. Probably the easiest way forward.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113241495
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -21,84 +21,90 @@
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collections;
    -import java.util.HashMap;
     import java.util.HashSet;
     import java.util.Map;
     import java.util.Set;
    -import java.util.function.Predicate;
     
    +import org.apache.accumulo.core.client.AccumuloException;
     import org.apache.accumulo.core.conf.Property;
     import org.apache.accumulo.core.volume.Volume;
     import org.apache.accumulo.server.client.HdfsZooInstance;
     import org.apache.accumulo.server.conf.ServerConfigurationFactory;
    -import org.apache.accumulo.server.conf.TableConfiguration;
     import org.apache.commons.collections.map.LRUMap;
     import org.apache.commons.lang.StringUtils;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX + "preferredVolumes";
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
           return super.choose(env, options);
    +    }
    +    ServerConfigurationFactory localConf = loadConf();
     
    -    // Get the current table's properties, and find the preferred volumes property
    -    // 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;
    +    String systemPreferredVolumes = localConf.getConfiguration().get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    if (null == systemPreferredVolumes || systemPreferredVolumes.isEmpty()) {
    +      String logMessage = "Default preferred volumes are missing.";
    +      log.debug(logMessage);
    +      throw new AccumuloException(logMessage);
         }
    -    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -    final Map<String,String> props = new HashMap<>();
    -    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
    -    if (props.isEmpty()) {
    -      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
    -      return super.choose(env, options);
    +
    +    String volumes = null;
    +    if (env.hasTableId()) {
    +      volumes = localConf.getTableConfiguration(env.getTableId()).get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    } else if (env.hasScope()) {
    +      volumes = localConf.getConfiguration().get(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".preferredVolumes");
    +    }
    +
    +    // if there was an empty or missing property, use the system-wide volumes
    +    if (null == volumes || volumes.isEmpty()) {
    +      if (env.hasTableId()) {
    +        log.warn("Missing property for TableID " + env.getTableId() + " but it should have picked up default volumes.");
    +      } else {
    +        log.debug("Missing preferred volumes for scope " + env.getScope() + ". Using default volumes.");
    --- End diff --
    
    Why is this debug?  Should the expected property name be in the message?  Should this fail instead of log?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r119186512
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -67,86 +69,67 @@ public String choose(VolumeChooserEnvironment env, String[] options) throws Accu
         return chooser.choose(env, options);
       }
     
    -  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
         if (log.isTraceEnabled()) {
           log.trace("Table id: " + env.getTableId());
         }
         final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
         String clazz = tableConf.get(Property.TABLE_VOLUME_CHOOSER);
    -    if (null == clazz || clazz.isEmpty()) {
    -      chooser = fallbackVolumeChooser;
    -    } else {
    -      chooser = tableSpecificChooser.get(env.getTableId());
    -      if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    -        if (chooser == null) {
    -          chooser = temp;
    -          // Otherwise, someone else beat us to initializing; use theirs.
    -        }
    -      } else if (!(chooser.getClass().getName().equals(clazz))) {
    -        if (log.isTraceEnabled()) {
    -          log.trace("change detected for table id: " + env.getTableId());
    -        }
    -        // the configuration for this table's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
    -        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        VolumeChooser last = tableSpecificChooser.replace(env.getTableId(), temp);
    -        if (chooser.equals(last)) {
    -          chooser = temp;
    -        } else {
    -          // Someone else beat us to updating; use theirs.
    -          chooser = last;
    -        }
    -      }
    -    }
    -    return chooser;
    +
    +    return createVolumeChooser(clazz, Property.TABLE_VOLUME_CHOOSER.getKey(), env.getTableId(), tableSpecificChooser);
       }
     
    -  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
    -    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +    String property = VOLUME_CHOOSER_SCOPED_KEY(env.getScope());
     
         if (log.isTraceEnabled()) {
           log.trace("Scope: " + env.getScope());
    -      log.trace("Looking up property: " + customProperty);
    +      log.trace("Looking up property: " + property);
         }
     
         AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    -    String clazz = systemConfiguration.get(customProperty);
    +    String clazz = systemConfiguration.get(property);
    +    // only if the custom property is not set to we fallback to the table volumn chooser setting
    +    if (null == clazz) {
    --- End diff --
    
    @ctubbsii The problem is that in the PerTableVolumeChooser we are looking for general.custom.logger.volume.chooser, and then falling back to the table.volume.chooser if not set (see lines 93-97).  This seems a like hokey.  So instead we are thinking of changing the property to tserver.nontable.volume.chooser (non-scope specific) and then remove the fallback to the table.volume.chooser.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by matthpeterson <gi...@git.apache.org>.
Github user matthpeterson commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r119147394
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -67,86 +69,67 @@ public String choose(VolumeChooserEnvironment env, String[] options) throws Accu
         return chooser.choose(env, options);
       }
     
    -  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
         if (log.isTraceEnabled()) {
           log.trace("Table id: " + env.getTableId());
         }
         final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
         String clazz = tableConf.get(Property.TABLE_VOLUME_CHOOSER);
    -    if (null == clazz || clazz.isEmpty()) {
    -      chooser = fallbackVolumeChooser;
    -    } else {
    -      chooser = tableSpecificChooser.get(env.getTableId());
    -      if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    -        if (chooser == null) {
    -          chooser = temp;
    -          // Otherwise, someone else beat us to initializing; use theirs.
    -        }
    -      } else if (!(chooser.getClass().getName().equals(clazz))) {
    -        if (log.isTraceEnabled()) {
    -          log.trace("change detected for table id: " + env.getTableId());
    -        }
    -        // the configuration for this table's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
    -        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        VolumeChooser last = tableSpecificChooser.replace(env.getTableId(), temp);
    -        if (chooser.equals(last)) {
    -          chooser = temp;
    -        } else {
    -          // Someone else beat us to updating; use theirs.
    -          chooser = last;
    -        }
    -      }
    -    }
    -    return chooser;
    +
    +    return createVolumeChooser(clazz, Property.TABLE_VOLUME_CHOOSER.getKey(), env.getTableId(), tableSpecificChooser);
       }
     
    -  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
    -    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +    String property = VOLUME_CHOOSER_SCOPED_KEY(env.getScope());
     
         if (log.isTraceEnabled()) {
           log.trace("Scope: " + env.getScope());
    -      log.trace("Looking up property: " + customProperty);
    +      log.trace("Looking up property: " + property);
         }
     
         AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    -    String clazz = systemConfiguration.get(customProperty);
    +    String clazz = systemConfiguration.get(property);
    +    // only if the custom property is not set to we fallback to the table volumn chooser setting
    +    if (null == clazz) {
    --- End diff --
    
    Now that we're failing on missing or bad configurations, it seems odd to assume that a missing wa logger volume manager should automatically use the system-level table volume manager.  It seems we should fail if this configuration is missing.  In terms of documenting the required properties for PerTableVolumeChooser, it should require both the table property and one for the logger.  Given that there is only one non-table scope at the moment, perhaps it makes sense to just have the single non-table property instead of including a scope in the property name.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113243239
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/conf/Property.java ---
    @@ -191,7 +191,7 @@
           + "server-internal scheduled tasks"),
       // If you update the default type, be sure to update the default used for initialization failures in VolumeManagerImpl
       @Experimental
    -  GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.PerTableVolumeChooser", PropertyType.CLASSNAME,
    +  GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.RandomVolumeChooser", PropertyType.CLASSNAME,
    --- End diff --
    
    Why do the defaults need to change?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129890801
  
    --- Diff: assemble/conf/accumulo-site.xml ---
    @@ -39,6 +39,15 @@
         <name>tserver.memory.maps.native.enabled</name>
         <value>true</value>
       </property>
    +  <!-- Used by the PerTableVolumeChooser which is the default general.volume.chooser -->
    --- End diff --
    
    sorry, that would be the general.volume.chooser


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129967127
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java ---
    @@ -28,5 +29,5 @@
      * benefit from using per-table configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
      */
     public interface VolumeChooser {
    -  String choose(VolumeChooserEnvironment env, String[] options);
    +  String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException;
    --- End diff --
    
    I could, but I think allowing one to throw accumulo exceptions is reasonable   Implementers do not need to include a throws clause.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129917175
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java ---
    @@ -79,8 +80,15 @@ protected VolumeManagerImpl(Map<String,Volume> volumes, Volume defaultVolume, Ac
         this.volumesByFileSystemUri = HashMultimap.create();
         invertVolumesByFileSystem(volumesByName, volumesByFileSystemUri);
         ensureSyncIsEnabled();
    -    // Keep in sync with default type in the property definition.
    -    chooser = Property.createInstanceFromPropertyName(conf, Property.GENERAL_VOLUME_CHOOSER, VolumeChooser.class, new PerTableVolumeChooser());
    +    // if they supplied a property and we cannot load it, then fail hard
    +    try {
    +      chooser = Property.createInstanceFromPropertyName(conf, Property.GENERAL_VOLUME_CHOOSER, VolumeChooser.class, null);
    +    } catch (NullPointerException npe) {
    --- End diff --
    
    Can this NPE check be consolidated with the null check later?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129960880
  
    --- 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()) {
    --- End diff --
    
    ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129964936
  
    --- 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 --
    
    I think I'm just second-guessing the basis for falling back in this way to a general-purpose "catch-all" for all scopes. It seems antithetical to the "fail hard if bad config" paradigm we discussed previously. I think there was a reason for it (backwards compatibility), but I'm not sure it's strictly necessary, since use of per-table volume choosing was always experimental.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129964201
  
    --- 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
    --- End diff --
    
    I can look into the enum change.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129895892
  
    --- 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 --
    
    Setting `property = ...` is going to change the error message in `createVolumeChooser`, so that now the user is going to be told to set the default catch-all for all scopes, rather than the scope-specific property.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129969789
  
    --- Diff: server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java ---
    @@ -105,7 +115,8 @@ public String choose(VolumeChooserEnvironment env, String[] options) {
       @SuppressWarnings("deprecation")
       private static final Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI;
     
    -  @Test
    +  // Expected to throw a runtime exception when the WrongVolumeChooser picks an invalid volume.
    +  @Test(expected = RuntimeException.class)
    --- End diff --
    
    ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129970246
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java ---
    @@ -0,0 +1,146 @@
    +/*
    + * 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
    + *
    + *     http://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.test;
    +
    +import java.io.File;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.SortedSet;
    +import java.util.TreeSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.Connector;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
    +import org.apache.accumulo.server.fs.PerTableVolumeChooser;
    +import org.apache.accumulo.server.fs.PreferredVolumeChooser;
    +import org.apache.accumulo.test.functional.ConfigurableMacBase;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.RawLocalFileSystem;
    +import org.apache.hadoop.io.Text;
    +import org.junit.Test;
    +
    +public class VolumeChooserFailureIT extends ConfigurableMacBase {
    --- End diff --
    
    there is javadoc on each of the actual @Test methods describing what is being tested.  is this not enough?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129918530
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java ---
    @@ -0,0 +1,146 @@
    +/*
    + * 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
    + *
    + *     http://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.test;
    +
    +import java.io.File;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.SortedSet;
    +import java.util.TreeSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.Connector;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
    +import org.apache.accumulo.server.fs.PerTableVolumeChooser;
    +import org.apache.accumulo.server.fs.PreferredVolumeChooser;
    +import org.apache.accumulo.test.functional.ConfigurableMacBase;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.RawLocalFileSystem;
    +import org.apache.hadoop.io.Text;
    +import org.junit.Test;
    +
    +public class VolumeChooserFailureIT extends ConfigurableMacBase {
    --- End diff --
    
    Needs javadoc explaining what exactly is being tested here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    I spoke with Matt, who wrote the original code for this PR, to get a sense of his design intentions. He clarified that the overall goal was to try to avoid `RandomVolumeChooser` as much as possible, no matter what went wrong (typos, etc.) when a user mis-typed a per-table chooser class or the class path was not set up correctly to find the user's chooser class.
    
    Currently (without this PR), Accumulo has hard-coded defaults on `RandomVolumeChooser` in several places. This can be problematic, because it can result in filling up volumes reserved for certain tables or the WALs. Further, it seems to select the `RandomVolumeChooser` mostly silently.
    
    This PR changes things so that if the user has configured the `PerTableVolumeChooser`, then when that fails to load a particular table's chooser (`table.volume.chooser` from a `TableConfiguration`), it will not fall back to the `RandomVolumeChooser`, but instead fall back to the system-wide setting (`SystemConfiguration`) for `table.volume.chooser` (for example, appearing in `accumulo-site.xml`, outside of any per-table context). If the user did not specify a `table.volume.chooser`, then then the `PerTableVolumeChooser` would simply fail. This ensures that `RandomVolumeChooser` is never used accidentally when using the `PerTableVolumeChooser`. A consequence of the change in this PR is that the defaults have changed, but this will only affect users who were relying on `general.volume.chooser` being the default and overriding `table.volume.chooser` (or vice-versa).
    
    The following is how I'm thinking it should probably work:
    Use the `RandomVolumeChooser` as the new default for `general.volume.chooser` and empty string (or null) for the default `table.volume.chooser`, as they are in this PR (see last sentence of previous paragraph). Document *VERY WELL* (especially in the release notes) that if the `PerTableVolumeChooser` is required by the user (e.g. they were relying on it being the default), then the user **must** set the `general.volume.chooser` to the `PerTableVolumeChooser` **AND** must either set a default value for `table.volume.chooser` in the system-wide configuration (system-wide ZK settings, or site file) **OR** ensure `table.volume.chooser` is set for *every* table (or namespace). If loading a per-table chooser or the general chooser fails at any point, we simply throw an exception, rather than trying to find a fall-back chooser to use.
    
    If we simply fail, rather than trying to fall back to some system-wide working configuration, we can still run into problems loading the class defined at that level, and it's not clear that is what the user wishes to occur. So, we should just fail instead. This would simplify this PR a bit.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113294827
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -21,84 +21,90 @@
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collections;
    -import java.util.HashMap;
     import java.util.HashSet;
     import java.util.Map;
     import java.util.Set;
    -import java.util.function.Predicate;
     
    +import org.apache.accumulo.core.client.AccumuloException;
     import org.apache.accumulo.core.conf.Property;
     import org.apache.accumulo.core.volume.Volume;
     import org.apache.accumulo.server.client.HdfsZooInstance;
     import org.apache.accumulo.server.conf.ServerConfigurationFactory;
    -import org.apache.accumulo.server.conf.TableConfiguration;
     import org.apache.commons.collections.map.LRUMap;
     import org.apache.commons.lang.StringUtils;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX + "preferredVolumes";
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
           return super.choose(env, options);
    +    }
    +    ServerConfigurationFactory localConf = loadConf();
     
    -    // Get the current table's properties, and find the preferred volumes property
    -    // 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;
    +    String systemPreferredVolumes = localConf.getConfiguration().get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    if (null == systemPreferredVolumes || systemPreferredVolumes.isEmpty()) {
    +      String logMessage = "Default preferred volumes are missing.";
    +      log.debug(logMessage);
    +      throw new AccumuloException(logMessage);
         }
    -    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -    final Map<String,String> props = new HashMap<>();
    -    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
    -    if (props.isEmpty()) {
    -      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
    -      return super.choose(env, options);
    +
    +    String volumes = null;
    +    if (env.hasTableId()) {
    +      volumes = localConf.getTableConfiguration(env.getTableId()).get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    } else if (env.hasScope()) {
    +      volumes = localConf.getConfiguration().get(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".preferredVolumes");
    +    }
    +
    +    // if there was an empty or missing property, use the system-wide volumes
    +    if (null == volumes || volumes.isEmpty()) {
    +      if (env.hasTableId()) {
    +        log.warn("Missing property for TableID " + env.getTableId() + " but it should have picked up default volumes.");
    --- End diff --
    
    It's probably a warning, because the current behavior falls back, but I think we'd want to change this so it's a hard fail. It seems it would only happen when there's a misconfiguration (or... without atomic config updates, if one property propagates via ZK faster than the corresponding properties).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    @ivakegg The test cases hung for me in Eclipse. I didn't try on the command-line yet. I'm still trying to understand what the tests are actually trying to do, and how we should change them if we switch to hard fail.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113244305
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -68,10 +100,107 @@ public String choose(VolumeChooserEnvironment env, String[] options) {
               chooser = last;
             }
           }
    +    }
    +    return chooser;
    +  }
    +
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +    VolumeChooser chooser;
    +    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +
    +    if (log.isTraceEnabled()) {
    +      log.trace("Scope: " + env.getScope());
    +      log.trace("Looking up property: " + customProperty);
    +    }
    +
    +    AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    +    String clazz = systemConfiguration.get(customProperty);
    +    if (null == clazz || clazz.isEmpty()) {
    +      log.debug("Property not found: " + customProperty + " using fallback chooser.");
    +      return fallbackVolumeChooser;
         } else {
    -      chooser = fallbackVolumeChooser;
    +      chooser = scopeSpecificChooser.get(env.getScope());
    +      if (chooser == null) {
    +        VolumeChooser temp;
    +        try {
    +          temp = loadClassForCustomProperty(clazz);
    +        } catch (Exception e) {
    +          log.error("Failed to create instance for " + env.getScope() + " configured to use " + clazz + " via " + customProperty);
    +          return fallbackVolumeChooser;
    --- End diff --
    
    Should we not be failing here instead of falling back?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    I rebase'd again, and updated to add an enum for scopes and to simplify VolumeChooserEnvironment a bit. Other work that needs to be completed:
    
    * per-table scope needs to use the table context for class loading.
    * need to fail if volume chosen is not in instance.volumes (it should always choose among the configured volumes)
    * need to support a global default in case per-table or scope-specific property is not set anywhere. if it is set, and fails to load, then it should simply fail. if it's not set, then it should try the global default (using the ChooserScope.DEFAULT and whatever property corresponds to that scope; property name depends on the chooser)
    * clean up / simplify property names


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129961801
  
    --- 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);
    --- End diff --
    
    I like that idea...shall do


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129966848
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -39,73 +40,139 @@
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String TABLE_PREFERRED_VOLUMES = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "preferred.volumes";
    +
    +  public static final String SCOPED_PREFERRED_VOLUMES(String scope) {
    +    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope + ".preferred.volumes";
    +  }
    +
    +  public static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = SCOPED_PREFERRED_VOLUMES("scoped");
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
           return super.choose(env, options);
    +    }
     
    -    // Get the current table's properties, and find the preferred volumes property
    -    // 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;
    +    // get the volumes property
    +    ServerConfigurationFactory localConf = loadConf();
    +    List<String> volumes;
    +    if (env.hasTableId()) {
    +      volumes = getPreferredVolumesForTable(env, localConf, options);
    +      // localConf.getTableConfiguration(env.getTableId()).get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    } else { // env.hasScope()
    +      volumes = getPreferredVolumesForNonTable(env, localConf, options);
    +      // localConf.getSystemConfiguration().get(PREFERRED_VOLUMES_SCOPED_KEY(env.getScope()));
         }
    -    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -    final Map<String,String> props = new HashMap<>();
    -    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
    -    if (props.isEmpty()) {
    -      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
    -      return super.choose(env, options);
    +
    +    // Randomly choose the volume from the preferred volumes
    +    String choice = super.choose(env, volumes.toArray(EMPTY_STRING_ARRAY));
    +    if (log.isTraceEnabled()) {
    +      log.trace("Choice = " + choice);
         }
    -    String volumes = props.get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    return choice;
    +  }
     
    +  private List<String> getPreferredVolumesForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf, String[] options)
    +      throws AccumuloException {
         if (log.isTraceEnabled()) {
    -      log.trace("In custom chooser");
    -      log.trace("Volumes: " + volumes);
    -      log.trace("TableID: " + env.getTableId());
    +      log.trace("Looking up property " + TABLE_PREFERRED_VOLUMES + " for Table id: " + env.getTableId());
         }
    -    // If the preferred volumes property was specified, split the returned string by the comma and add use it to filter the given options.
    -    Set<String> preferred = parsedPreferredVolumes.get(volumes);
    -    if (preferred == null) {
    -      preferred = new HashSet<>(Arrays.asList(StringUtils.split(volumes, ',')));
    -      parsedPreferredVolumes.put(volumes, preferred);
    +    final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    +    String volumes = tableConf.get(TABLE_PREFERRED_VOLUMES);
    +
    +    return getFilteredOptions(TABLE_PREFERRED_VOLUMES, volumes, options);
    +  }
    +
    +  private List<String> getPreferredVolumesForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf, String[] options)
    +      throws AccumuloException {
    +    String property = SCOPED_PREFERRED_VOLUMES(env.getScope());
    +
    +    if (log.isTraceEnabled()) {
    +      log.trace("Looking up property: " + property);
    +    }
    +
    +    AccumuloConfiguration systemConfiguration = localConf.getSystemConfiguration();
    +    String volumes = systemConfiguration.get(property);
    +
    +    // only if the custom property is not set to we fallback to the default scoped preferred volumes
    +    if (null == volumes) {
    +      log.debug("Property not found: " + property + " using " + DEFAULT_SCOPED_PREFERRED_VOLUMES);
    +      property = DEFAULT_SCOPED_PREFERRED_VOLUMES;
    +      volumes = systemConfiguration.get(DEFAULT_SCOPED_PREFERRED_VOLUMES);
    --- End diff --
    
    If we want this to work like the table preferred volume where you can set a default in the system configuration, the we would need to have scoped configurations like we have table specific configurations.  I am not sure how to handle this in the same way.  We could get rid of the default scope property but then we would be requiring users to set both the "logger" and "init" scoped properties which seems wrong.  Realistically users will only set the default one until we actually come up with other scopes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r119166010
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -67,86 +69,67 @@ public String choose(VolumeChooserEnvironment env, String[] options) throws Accu
         return chooser.choose(env, options);
       }
     
    -  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
         if (log.isTraceEnabled()) {
           log.trace("Table id: " + env.getTableId());
         }
         final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
         String clazz = tableConf.get(Property.TABLE_VOLUME_CHOOSER);
    -    if (null == clazz || clazz.isEmpty()) {
    -      chooser = fallbackVolumeChooser;
    -    } else {
    -      chooser = tableSpecificChooser.get(env.getTableId());
    -      if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    -        if (chooser == null) {
    -          chooser = temp;
    -          // Otherwise, someone else beat us to initializing; use theirs.
    -        }
    -      } else if (!(chooser.getClass().getName().equals(clazz))) {
    -        if (log.isTraceEnabled()) {
    -          log.trace("change detected for table id: " + env.getTableId());
    -        }
    -        // the configuration for this table's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
    -        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        VolumeChooser last = tableSpecificChooser.replace(env.getTableId(), temp);
    -        if (chooser.equals(last)) {
    -          chooser = temp;
    -        } else {
    -          // Someone else beat us to updating; use theirs.
    -          chooser = last;
    -        }
    -      }
    -    }
    -    return chooser;
    +
    +    return createVolumeChooser(clazz, Property.TABLE_VOLUME_CHOOSER.getKey(), env.getTableId(), tableSpecificChooser);
       }
     
    -  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
    -    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +    String property = VOLUME_CHOOSER_SCOPED_KEY(env.getScope());
     
         if (log.isTraceEnabled()) {
           log.trace("Scope: " + env.getScope());
    -      log.trace("Looking up property: " + customProperty);
    +      log.trace("Looking up property: " + property);
         }
     
         AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    -    String clazz = systemConfiguration.get(customProperty);
    +    String clazz = systemConfiguration.get(property);
    +    // only if the custom property is not set to we fallback to the table volumn chooser setting
    +    if (null == clazz) {
    --- End diff --
    
    I like that idea.  Perhaps in addition to table.volume.chooser we have a tserver.nontable.volume.chooser.  Any other thoughts as to the property name?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129972561
  
    --- Diff: test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java ---
    @@ -0,0 +1,146 @@
    +/*
    + * 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
    + *
    + *     http://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.test;
    +
    +import java.io.File;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.SortedSet;
    +import java.util.TreeSet;
    +
    +import org.apache.accumulo.core.client.AccumuloException;
    +import org.apache.accumulo.core.client.AccumuloSecurityException;
    +import org.apache.accumulo.core.client.Connector;
    +import org.apache.accumulo.core.client.TableNotFoundException;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
    +import org.apache.accumulo.server.fs.PerTableVolumeChooser;
    +import org.apache.accumulo.server.fs.PreferredVolumeChooser;
    +import org.apache.accumulo.test.functional.ConfigurableMacBase;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.RawLocalFileSystem;
    +import org.apache.hadoop.io.Text;
    +import org.junit.Test;
    +
    +public class VolumeChooserFailureIT extends ConfigurableMacBase {
    --- End diff --
    
    Okay, upon first reading, I didn't quite get the overall context. After reading them a few times, I get that the overall context is to test for expected failures when configuration is not fully satisfied. I just needed to read more carefully :smile:


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129890374
  
    --- Diff: assemble/conf/accumulo-site.xml ---
    @@ -39,6 +39,15 @@
         <name>tserver.memory.maps.native.enabled</name>
         <value>true</value>
       </property>
    +  <!-- Used by the PerTableVolumeChooser which is the default general.volume.chooser -->
    --- End diff --
    
    Agreed....I will change the default for the table.volume.chooser


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129969952
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java ---
    @@ -28,5 +29,5 @@
      * benefit from using per-table configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
      */
     public interface VolumeChooser {
    -  String choose(VolumeChooserEnvironment env, String[] options);
    +  String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException;
    --- End diff --
    
    AccumuloException is really a client-side exception, and probably shouldn't be used internally on the server-side.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113294170
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -18,36 +18,65 @@
     
     import java.util.concurrent.ConcurrentHashMap;
     
    +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}.
      */
     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<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;
    +  private volatile VolumeChooser fallbackVolumeChooser = null;
     
       @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;
    -      }
    -      final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    +  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();
    +    lazilyCreateFallbackChooser();
    +    if (env.hasScope()) {
    --- End diff --
    
    `env.hasTableId()` and `env.hasScope()` are mutually exclusive right now. It's possible that could change, but we'd have to think about what that might mean. Currently, the only scope is "logger" and it has no table associated with it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r113241222
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java ---
    @@ -21,84 +21,90 @@
     import java.util.ArrayList;
     import java.util.Arrays;
     import java.util.Collections;
    -import java.util.HashMap;
     import java.util.HashSet;
     import java.util.Map;
     import java.util.Set;
    -import java.util.function.Predicate;
     
    +import org.apache.accumulo.core.client.AccumuloException;
     import org.apache.accumulo.core.conf.Property;
     import org.apache.accumulo.core.volume.Volume;
     import org.apache.accumulo.server.client.HdfsZooInstance;
     import org.apache.accumulo.server.conf.ServerConfigurationFactory;
    -import org.apache.accumulo.server.conf.TableConfiguration;
     import org.apache.commons.collections.map.LRUMap;
     import org.apache.commons.lang.StringUtils;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     /**
      * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset of those options preferred for a particular table. Defaults
    - * to selecting from all of the options presented. Can be customized via the table property {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
    + * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, which should contain a comma
      * separated list of {@link Volume} URIs. Note that both the property name and the format of its value are specific to this particular implementation.
      */
     public class PreferredVolumeChooser extends RandomVolumeChooser {
       private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
     
    -  /**
    -   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
    -   */
    -  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
    -  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
    -  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
    +  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX + "preferredVolumes";
     
       @SuppressWarnings("unchecked")
       private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
       // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
       private volatile ServerConfigurationFactory serverConfs;
     
       @Override
    -  public String choose(VolumeChooserEnvironment env, String[] options) {
    -    if (!env.hasTableId())
    +  public String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException {
    +    if (!env.hasTableId() && !env.hasScope()) {
    +      // this should only happen during initialize
    +      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
           return super.choose(env, options);
    +    }
    +    ServerConfigurationFactory localConf = loadConf();
     
    -    // Get the current table's properties, and find the preferred volumes property
    -    // 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;
    +    String systemPreferredVolumes = localConf.getConfiguration().get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    if (null == systemPreferredVolumes || systemPreferredVolumes.isEmpty()) {
    +      String logMessage = "Default preferred volumes are missing.";
    +      log.debug(logMessage);
    +      throw new AccumuloException(logMessage);
         }
    -    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -    final Map<String,String> props = new HashMap<>();
    -    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
    -    if (props.isEmpty()) {
    -      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
    -      return super.choose(env, options);
    +
    +    String volumes = null;
    +    if (env.hasTableId()) {
    +      volumes = localConf.getTableConfiguration(env.getTableId()).get(PREFERRED_VOLUMES_CUSTOM_KEY);
    +    } else if (env.hasScope()) {
    +      volumes = localConf.getConfiguration().get(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".preferredVolumes");
    +    }
    +
    +    // if there was an empty or missing property, use the system-wide volumes
    +    if (null == volumes || volumes.isEmpty()) {
    +      if (env.hasTableId()) {
    +        log.warn("Missing property for TableID " + env.getTableId() + " but it should have picked up default volumes.");
    --- End diff --
    
    Could include `table.custom.preferredVolumes` in the log message.
    
    Why log a warn instead of throw an exception here?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129897225
  
    --- 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;
    +      clazz = systemConfiguration.get(DEFAULT_SCOPED_VOLUME_CHOOSER);
    +    }
    +
    +    return createVolumeChooser(clazz, property, env.getScope(), scopeSpecificChooser);
    +  }
    +
    +  private VolumeChooser createVolumeChooser(String clazz, String property, String key, ConcurrentHashMap<String,VolumeChooser> cache) throws AccumuloException {
    +    if (null == clazz || clazz.isEmpty()) {
    +      String msg = "Property " + property + " must be set" + (null == clazz ? " " : " properly ") + "to use the " + getClass().getSimpleName();
    +      log.error(msg);
    +      throw new AccumuloException(msg);
    +    }
    +
    +    VolumeChooser chooser = cache.get(key);
    +    if (chooser == null || !(chooser.getClass().getName().equals(clazz))) {
    +      if (log.isTraceEnabled() && chooser != null) {
    +        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    +        log.trace("Change detected for " + property + " for " + key);
    +      }
    +      VolumeChooser temp;
    +      try {
    +        temp = loadClass(clazz);
    +      } catch (Exception e) {
    +        String msg = "Failed to create instance for " + key + " configured to use " + clazz + " via " + property;
    +        log.error(msg);
    +        throw new AccumuloException(msg, e);
           }
    -      final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
    -      chooser = tableSpecificChooser.get(env.getTableId());
           if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    +        chooser = cache.putIfAbsent(key, temp);
    --- End diff --
    
    If cache is synchronized, the check for null in next line seems moot.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r130088538
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java ---
    @@ -28,5 +29,5 @@
      * benefit from using per-table configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
      */
     public interface VolumeChooser {
    -  String choose(VolumeChooserEnvironment env, String[] options);
    +  String choose(VolumeChooserEnvironment env, String[] options) throws AccumuloException;
    --- End diff --
    
    ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129960835
  
    --- 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<>();
    --- End diff --
    
    You will see that the same routine (createVolumeChooser) is used to add volume choosers into those maps, so I had to use a common denominator for the class. String works well. However if you insist that I use Table.ID here instead, then I will denormalize the common method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r130191638
  
    --- 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);
    --- End diff --
    
    Thanks, @ivakegg !


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    Do the test cases hang when run via mvn?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r130157740
  
    --- 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);
    --- End diff --
    
    Would it be possible to code init as though it were a proper scope, but not make it configurable? (e.g. always return random chooser if scope == init?)
    
    I'm also thinking that instead of it being "scope || per-table", we have "per-table" treated like a proper scope itself. So, `if scope == per-table: choosePerTable(env.table)`
    
    I can work on this slight refactoring if you're done with your bits, as it's related to the enums we previously discussed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129962182
  
    --- 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
    --- End diff --
    
    oops, missed that comment.
    re: scopes.  If the scopes were in an enum then this might be cleaner.  I will look into that but perhaps that should be a separate ticket.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129968127
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java ---
    @@ -79,8 +80,15 @@ protected VolumeManagerImpl(Map<String,Volume> volumes, Volume defaultVolume, Ac
         this.volumesByFileSystemUri = HashMultimap.create();
         invertVolumesByFileSystem(volumesByName, volumesByFileSystemUri);
         ensureSyncIsEnabled();
    -    // Keep in sync with default type in the property definition.
    -    chooser = Property.createInstanceFromPropertyName(conf, Property.GENERAL_VOLUME_CHOOSER, VolumeChooser.class, new PerTableVolumeChooser());
    +    // if they supplied a property and we cannot load it, then fail hard
    +    try {
    +      chooser = Property.createInstanceFromPropertyName(conf, Property.GENERAL_VOLUME_CHOOSER, VolumeChooser.class, null);
    +    } catch (NullPointerException npe) {
    --- End diff --
    
    yes


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    @ivakegg  Rather than merge your last PR, I simply rebased, and resolved the errors, and cleaned up some unused variables. Have you been able to get VolumeChooserForNonTableScopeIT to pass? It always hangs when I try.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r119186725
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java ---
    @@ -67,86 +69,67 @@ public String choose(VolumeChooserEnvironment env, String[] options) throws Accu
         return chooser.choose(env, options);
       }
     
    -  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
         if (log.isTraceEnabled()) {
           log.trace("Table id: " + env.getTableId());
         }
         final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
         String clazz = tableConf.get(Property.TABLE_VOLUME_CHOOSER);
    -    if (null == clazz || clazz.isEmpty()) {
    -      chooser = fallbackVolumeChooser;
    -    } else {
    -      chooser = tableSpecificChooser.get(env.getTableId());
    -      if (chooser == null) {
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
    -        if (chooser == null) {
    -          chooser = temp;
    -          // Otherwise, someone else beat us to initializing; use theirs.
    -        }
    -      } else if (!(chooser.getClass().getName().equals(clazz))) {
    -        if (log.isTraceEnabled()) {
    -          log.trace("change detected for table id: " + env.getTableId());
    -        }
    -        // the configuration for this table's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
    -        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
    -        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
    -        VolumeChooser last = tableSpecificChooser.replace(env.getTableId(), temp);
    -        if (chooser.equals(last)) {
    -          chooser = temp;
    -        } else {
    -          // Someone else beat us to updating; use theirs.
    -          chooser = last;
    -        }
    -      }
    -    }
    -    return chooser;
    +
    +    return createVolumeChooser(clazz, Property.TABLE_VOLUME_CHOOSER.getKey(), env.getTableId(), tableSpecificChooser);
       }
     
    -  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
    +  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) throws AccumuloException {
         VolumeChooser chooser;
    -    final String customProperty = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + env.getScope() + ".volume.chooser";
    +    String property = VOLUME_CHOOSER_SCOPED_KEY(env.getScope());
     
         if (log.isTraceEnabled()) {
           log.trace("Scope: " + env.getScope());
    -      log.trace("Looking up property: " + customProperty);
    +      log.trace("Looking up property: " + property);
         }
     
         AccumuloConfiguration systemConfiguration = localConf.getConfiguration();
    -    String clazz = systemConfiguration.get(customProperty);
    +    String clazz = systemConfiguration.get(property);
    +    // only if the custom property is not set to we fallback to the table volumn chooser setting
    +    if (null == clazz) {
    --- End diff --
    
    @ctubbsii Please suggest other property names to use if you do not like the "baked-in" ones.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on the issue:

    https://github.com/apache/accumulo/pull/253
  
    As it turns out the VolumeChooserForNonTableScopeIT is no longer a valid test.  It is explicitly configuring to use the PreferredVolumeChooser but is not setting the volumes leaving the system badly configured.  Hence when the MiniAccumulo is started up and a table is created, it promptly falls over leaving the test case hanging on the createTable call.  I will try to figure out if I can get the test case to detect the failure in the MiniAccumulo.  If that is not possible then I will simply remove this test.  I think I need to fix up VolumeChooserIT.java as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/253#discussion_r129918169
  
    --- Diff: server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java ---
    @@ -105,7 +115,8 @@ public String choose(VolumeChooserEnvironment env, String[] options) {
       @SuppressWarnings("deprecation")
       private static final Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI;
     
    -  @Test
    +  // Expected to throw a runtime exception when the WrongVolumeChooser picks an invalid volume.
    +  @Test(expected = RuntimeException.class)
    --- End diff --
    
    Using ExpectedException JUnit rule allows more fine checking of the thrown exception.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---