Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 410C1200CD3 for ; Fri, 28 Jul 2017 20:37:44 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 3F83116D5C8; Fri, 28 Jul 2017 18:37:44 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 849F216D5C4 for ; Fri, 28 Jul 2017 20:37:43 +0200 (CEST) Received: (qmail 94464 invoked by uid 500); 28 Jul 2017 18:37:36 -0000 Mailing-List: contact dev-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list dev@accumulo.apache.org Received: (qmail 94453 invoked by uid 99); 28 Jul 2017 18:37:36 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 28 Jul 2017 18:37:36 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 631A5E02ED; Fri, 28 Jul 2017 18:37:36 +0000 (UTC) From: ctubbsii To: dev@accumulo.apache.org Reply-To: dev@accumulo.apache.org References: In-Reply-To: Subject: [GitHub] accumulo pull request #253: ACCUMULO-4086 Improve volume chooser fallback Content-Type: text/plain Message-Id: <20170728183736.631A5E02ED@git1-us-west.apache.org> Date: Fri, 28 Jul 2017 18:37:36 +0000 (UTC) archived-at: Fri, 28 Jul 2017 18:37:44 -0000 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 tableSpecificChooser = new ConcurrentHashMap<>(); + private final ConcurrentHashMap tableSpecificChooser = new ConcurrentHashMap<>(); + private final ConcurrentHashMap 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. ---