accumulo-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ivakegg <...@git.apache.org>
Subject [GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback
Date Fri, 28 Jul 2017 15:44:08 GMT
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.
---

Mime
View raw message