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 468BF200D3A for ; Wed, 11 Oct 2017 02:29:16 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 45255160BE0; Wed, 11 Oct 2017 00:29:16 +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 B5451160BE5 for ; Wed, 11 Oct 2017 02:29:13 +0200 (CEST) Received: (qmail 83682 invoked by uid 500); 11 Oct 2017 00:29:12 -0000 Mailing-List: contact commits-help@aurora.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@aurora.apache.org Delivered-To: mailing list commits@aurora.apache.org Received: (qmail 83512 invoked by uid 99); 11 Oct 2017 00:29:12 -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; Wed, 11 Oct 2017 00:29:12 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 9DB14F5D10; Wed, 11 Oct 2017 00:29:10 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wfarner@apache.org To: commits@aurora.apache.org Date: Wed, 11 Oct 2017 00:29:13 -0000 Message-Id: <7e3e298949b748048004cb7f053eb8f1@git.apache.org> In-Reply-To: <267bd21f25a441cca38d85834ead8d5b@git.apache.org> References: <267bd21f25a441cca38d85834ead8d5b@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [4/8] aurora git commit: Use a simpler command line argument system archived-at: Wed, 11 Oct 2017 00:29:16 -0000 http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/app/VolumeConverter.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/app/VolumeConverter.java b/src/main/java/org/apache/aurora/scheduler/app/VolumeConverter.java new file mode 100644 index 0000000..d8af1f9 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/app/VolumeConverter.java @@ -0,0 +1,54 @@ +/** + * Licensed 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.aurora.scheduler.app; + +import com.beust.jcommander.ParameterException; +import com.beust.jcommander.converters.BaseConverter; +import com.google.common.base.Joiner; + +import org.apache.aurora.gen.Mode; +import org.apache.aurora.gen.Volume; + +/** + * Converter to transform a string in host:container:mode form to a VolumeConfig object. + */ +public class VolumeConverter extends BaseConverter { + + public VolumeConverter() { + super(""); + } + + public VolumeConverter(String optionName) { + super(optionName); + } + + @Override + public Volume convert(String raw) { + String[] split = raw.split(":"); + if (split.length != 3) { + throw new ParameterException( + getErrorString(raw, "must be in the format of 'host:container:mode'")); + } + + Mode mode; + try { + mode = Mode.valueOf(split[2].toUpperCase()); + } catch (IllegalArgumentException e) { + throw new ParameterException( + getErrorString(raw, "Read/Write spec must be in " + Joiner.on(", ").join(Mode.values())), + e); + } + return new Volume(split[1], split[0], mode); + } +} http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/app/VolumeParser.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/app/VolumeParser.java b/src/main/java/org/apache/aurora/scheduler/app/VolumeParser.java deleted file mode 100644 index c1e99ce..0000000 --- a/src/main/java/org/apache/aurora/scheduler/app/VolumeParser.java +++ /dev/null @@ -1,46 +0,0 @@ -/** - * Licensed 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.aurora.scheduler.app; - -import com.google.common.base.Joiner; - -import org.apache.aurora.common.args.ArgParser; -import org.apache.aurora.common.args.parsers.NonParameterizedTypeParser; -import org.apache.aurora.gen.Mode; -import org.apache.aurora.gen.Volume; - -/** - * Parser to transform a string in host:container:mode form to a VolumeConfig - * object. - */ -@ArgParser -public class VolumeParser extends NonParameterizedTypeParser { - @Override - public Volume doParse(String raw) throws IllegalArgumentException { - String[] split = raw.split(":"); - if (split.length != 3) { - throw new IllegalArgumentException("Illegal mount string " + raw + ". " - + "Mounts must be in the format of 'host:container:mode'"); - } - - Mode mode; - try { - mode = Mode.valueOf(split[2].toUpperCase()); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException("Illegal mount string " + raw + ". " - + "Read/Write spec must be in " + Joiner.on(", ").join(Mode.values()), e); - } - return new Volume(split[1], split[0], mode); - } -} http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java b/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java index da07df6..68f7ddb 100644 --- a/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java +++ b/src/main/java/org/apache/aurora/scheduler/async/AsyncModule.java @@ -23,13 +23,13 @@ import javax.inject.Inject; import javax.inject.Qualifier; import javax.inject.Singleton; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.AbstractIdleService; import com.google.inject.AbstractModule; import com.google.inject.PrivateModule; -import org.apache.aurora.common.args.Arg; -import org.apache.aurora.common.args.CmdLine; import org.apache.aurora.common.stats.StatsProvider; import org.apache.aurora.scheduler.SchedulerServicesModule; import org.apache.aurora.scheduler.base.AsyncUtil; @@ -48,19 +48,26 @@ import static java.util.Objects.requireNonNull; public class AsyncModule extends AbstractModule { private static final Logger LOG = LoggerFactory.getLogger(AsyncModule.class); - @CmdLine(name = "async_worker_threads", - help = "The number of worker threads to process async task operations with.") - private static final Arg ASYNC_WORKER_THREADS = Arg.create(8); + @Parameters(separators = "=") + public static class Options { + @Parameter(names = "-async_worker_threads", + description = "The number of worker threads to process async task operations with.") + public int asyncWorkerThreads = 8; + } + private final ScheduledThreadPoolExecutor afterTransaction; @Qualifier @Target({ FIELD, PARAMETER, METHOD }) @Retention(RUNTIME) public @interface AsyncExecutor { } - public AsyncModule() { + public AsyncModule(Options options) { // Don't worry about clean shutdown, these can be daemon and cleanup-free. // TODO(wfarner): Should we use a bounded caching thread pool executor instead? - this(AsyncUtil.loggingScheduledExecutor(ASYNC_WORKER_THREADS.get(), "AsyncProcessor-%d", LOG)); + this(AsyncUtil.loggingScheduledExecutor( + options.asyncWorkerThreads, + "AsyncProcessor-%d", + LOG)); } @VisibleForTesting http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java index 186fa1b..60bbe39 100644 --- a/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java +++ b/src/main/java/org/apache/aurora/scheduler/base/TaskTestUtil.java @@ -19,7 +19,6 @@ import java.util.Set; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.ImmutableSet; import org.apache.aurora.gen.AssignedTask; @@ -83,7 +82,7 @@ public final class TaskTestUtil { new ConfigurationManagerSettings( ImmutableSet.of(_Fields.MESOS), false, - ImmutableMultimap.of(), + ImmutableList.of(), true, true, true, http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/config/CliOptions.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/config/CliOptions.java b/src/main/java/org/apache/aurora/scheduler/config/CliOptions.java new file mode 100644 index 0000000..64733e5 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/config/CliOptions.java @@ -0,0 +1,114 @@ +/** + * Licensed 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.aurora.scheduler.config; + +import java.util.List; + +import com.google.common.base.Predicates; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; + +import org.apache.aurora.scheduler.SchedulerModule; +import org.apache.aurora.scheduler.TierModule; +import org.apache.aurora.scheduler.app.AppModule; +import org.apache.aurora.scheduler.app.SchedulerMain; +import org.apache.aurora.scheduler.async.AsyncModule; +import org.apache.aurora.scheduler.configuration.executor.ExecutorModule; +import org.apache.aurora.scheduler.cron.quartz.CronModule; +import org.apache.aurora.scheduler.discovery.FlaggedZooKeeperConfig; +import org.apache.aurora.scheduler.events.WebhookModule; +import org.apache.aurora.scheduler.http.H2ConsoleModule; +import org.apache.aurora.scheduler.http.JettyServerModule; +import org.apache.aurora.scheduler.http.api.ApiModule; +import org.apache.aurora.scheduler.http.api.security.HttpSecurityModule; +import org.apache.aurora.scheduler.http.api.security.IniShiroRealmModule; +import org.apache.aurora.scheduler.http.api.security.Kerberos5ShiroRealmModule; +import org.apache.aurora.scheduler.log.mesos.MesosLogStreamModule; +import org.apache.aurora.scheduler.mesos.CommandLineDriverSettingsModule; +import org.apache.aurora.scheduler.offers.OffersModule; +import org.apache.aurora.scheduler.preemptor.PreemptorModule; +import org.apache.aurora.scheduler.pruning.PruningModule; +import org.apache.aurora.scheduler.reconciliation.ReconciliationModule; +import org.apache.aurora.scheduler.resources.ResourceSettings; +import org.apache.aurora.scheduler.scheduling.SchedulingModule; +import org.apache.aurora.scheduler.sla.SlaModule; +import org.apache.aurora.scheduler.state.StateModule; +import org.apache.aurora.scheduler.stats.AsyncStatsModule; +import org.apache.aurora.scheduler.stats.StatsModule; +import org.apache.aurora.scheduler.storage.backup.BackupModule; +import org.apache.aurora.scheduler.storage.db.DbModule; +import org.apache.aurora.scheduler.storage.log.LogStorageModule; +import org.apache.aurora.scheduler.thrift.aop.AopModule; +import org.apache.aurora.scheduler.updater.UpdaterModule; + +public class CliOptions { + public final ReconciliationModule.Options reconciliation = + new ReconciliationModule.Options(); + public final OffersModule.Options offer = new OffersModule.Options(); + public final ExecutorModule.Options executor = new ExecutorModule.Options(); + public final AppModule.Options app = new AppModule.Options(); + public final SchedulerMain.Options main = new SchedulerMain.Options(); + public final SchedulingModule.Options scheduling = new SchedulingModule.Options(); + public final AsyncModule.Options async = new AsyncModule.Options(); + public final FlaggedZooKeeperConfig.Options zk = new FlaggedZooKeeperConfig.Options(); + public final UpdaterModule.Options updater = new UpdaterModule.Options(); + public final StateModule.Options state = new StateModule.Options(); + public final DbModule.Options db = new DbModule.Options(); + public final LogStorageModule.Options logStorage = new LogStorageModule.Options(); + public final BackupModule.Options backup = new BackupModule.Options(); + public final AopModule.Options aop = new AopModule.Options(); + public final PruningModule.Options pruning = new PruningModule.Options(); + public final CommandLineDriverSettingsModule.Options driver = + new CommandLineDriverSettingsModule.Options(); + public final JettyServerModule.Options jetty = new JettyServerModule.Options(); + public final HttpSecurityModule.Options httpSecurity = new HttpSecurityModule.Options(); + public final Kerberos5ShiroRealmModule.Options kerberos = new Kerberos5ShiroRealmModule.Options(); + public final IniShiroRealmModule.Options iniShiroRealm = new IniShiroRealmModule.Options(); + public final ApiModule.Options api = new ApiModule.Options(); + public final H2ConsoleModule.Options h2Console = new H2ConsoleModule.Options(); + public final PreemptorModule.Options preemptor = new PreemptorModule.Options(); + public final MesosLogStreamModule.Options mesosLog = new MesosLogStreamModule.Options(); + public final SlaModule.Options sla = new SlaModule.Options(); + public final WebhookModule.Options webhook = new WebhookModule.Options(); + public final SchedulerModule.Options scheduler = new SchedulerModule.Options(); + public final TierModule.Options tiers = new TierModule.Options(); + public final AsyncStatsModule.Options asyncStats = new AsyncStatsModule.Options(); + public final StatsModule.Options stats = new StatsModule.Options(); + public final CronModule.Options cron = new CronModule.Options(); + public final ResourceSettings resourceSettings = new ResourceSettings(); + final List custom; + + public CliOptions() { + this(ImmutableList.of()); + } + + public CliOptions(List custom) { + this.custom = custom; + } + + /** + * Gets a custom options object of a particular type. + * + * @param customOptionType Custom option type class. + * @param Custom option type. + * @return The matching custom option object. + */ + @SuppressWarnings("unchecked") + public T getCustom(Class customOptionType) { + return (T) FluentIterable.from(custom) + .firstMatch(Predicates.instanceOf(customOptionType)) + .get(); + } +} http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/config/CommandLine.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/config/CommandLine.java b/src/main/java/org/apache/aurora/scheduler/config/CommandLine.java new file mode 100644 index 0000000..2085810 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/config/CommandLine.java @@ -0,0 +1,198 @@ +/** + * Licensed 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.aurora.scheduler.config; + +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.net.InetSocketAddress; +import java.util.List; +import java.util.Map; + +import javax.security.auth.kerberos.KerberosPrincipal; + +import com.beust.jcommander.IStringConverter; +import com.beust.jcommander.IStringConverterFactory; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.ParameterException; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import org.apache.aurora.gen.DockerParameter; +import org.apache.aurora.gen.Volume; +import org.apache.aurora.scheduler.app.SchedulerMain; +import org.apache.aurora.scheduler.app.VolumeConverter; +import org.apache.aurora.scheduler.config.converters.ClassConverter; +import org.apache.aurora.scheduler.config.converters.DataAmountConverter; +import org.apache.aurora.scheduler.config.converters.DockerParameterConverter; +import org.apache.aurora.scheduler.config.converters.InetSocketAddressConverter; +import org.apache.aurora.scheduler.config.converters.TimeAmountConverter; +import org.apache.aurora.scheduler.config.types.DataAmount; +import org.apache.aurora.scheduler.config.types.TimeAmount; +import org.apache.aurora.scheduler.http.api.security.KerberosPrincipalConverter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Parses command line options and populates {@link CliOptions}. + */ +public final class CommandLine { + + private static final Logger LOG = LoggerFactory.getLogger(CommandLine.class); + + // TODO(wfarner): This can go away if/when options are no longer accessed statically. + private static CliOptions instance = null; + + private static List customOptions = Lists.newArrayList(); + + private CommandLine() { + // Utility class. + } + + /** + * Similar to {@link #initializeForTest()}, but resets the class to an un-parsed state. + */ + @VisibleForTesting + static void clearForTest() { + instance = null; + customOptions = Lists.newArrayList(); + } + + /** + * Initializes static command line state - the static parsed instance, and custom options objects. + */ + @VisibleForTesting + public static void initializeForTest() { + instance = new CliOptions(); + customOptions = Lists.newArrayList(); + } + + private static JCommander prepareParser(CliOptions options) { + JCommander.Builder builder = JCommander.newBuilder() + .programName(SchedulerMain.class.getName()); + + builder.addConverterFactory(new IStringConverterFactory() { + private Map, Class>> classConverters = + ImmutableMap., Class>>builder() + .put(Class.class, ClassConverter.class) + .put(DataAmount.class, DataAmountConverter.class) + .put(DockerParameter.class, DockerParameterConverter.class) + .put(InetSocketAddress.class, InetSocketAddressConverter.class) + .put(KerberosPrincipal.class, KerberosPrincipalConverter.class) + .put(TimeAmount.class, TimeAmountConverter.class) + .put(Volume.class, VolumeConverter.class) + .build(); + + @SuppressWarnings("unchecked") + @Override + public Class> getConverter(Class forType) { + return (Class>) classConverters.get(forType); + } + }); + + builder.addObject(getOptionsObjects(options)); + return builder.build(); + } + + /** + * Applies arg values to the options object. + * + * @param args Command line arguments. + */ + @VisibleForTesting + public static CliOptions parseOptions(String... args) { + JCommander parser = null; + try { + parser = prepareParser(new CliOptions(ImmutableList.copyOf(customOptions))); + + // We first perform a 'dummy' parsing round. This induces classloading on any third-party + // code, where they can statically invoke registerCustomOptions(). + parser.setAcceptUnknownOptions(true); + parser.parseWithoutValidation(args); + + CliOptions options = new CliOptions(ImmutableList.copyOf(customOptions)); + parser = prepareParser(options); + parser.parse(args); + instance = options; + return options; + } catch (ParameterException e) { + if (parser != null) { + parser.usage(); + } + LOG.error(e.getMessage()); + System.exit(1); + throw new RuntimeException(e); + } catch (RuntimeException e) { + throw e; + } + } + + /** + * Gets the static and globally-accessible CLI options. This exists only to support legacy use + * cases that cannot yet support injected arguments. New callers should not be added. + * + * @return global options + */ + public static CliOptions legacyGetStaticOptions() { + if (instance == null) { + throw new IllegalStateException("Attempted to fetch command line arguments before parsing."); + } + return instance; + } + + /** + * Registers a custom options container for inclusion during command line option parsing. This + * is useful to allow third-party modules to include custom command line options. + * + * @param options Custom options object. + * See {@link com.beust.jcommander.JCommander.Builder#addObject(Object)} for + * details. + */ + public static void registerCustomOptions(Object options) { + Preconditions.checkState( + instance == null, + "Attempted to register custom options after command line parsing."); + + customOptions.add(options); + } + + @VisibleForTesting + static List getOptionsObjects(CliOptions options) { + ImmutableList.Builder objects = ImmutableList.builder(); + + // Reflect on fields defined in CliOptions to DRY and avoid mistakes of forgetting to add an + // option field here. + for (Field field : CliOptions.class.getDeclaredFields()) { + if (Modifier.isStatic(field.getModifiers())) { + continue; + } + + try { + if (Iterable.class.isAssignableFrom(field.getType())) { + Iterable iterableValue = (Iterable) field.get(options); + objects.addAll(iterableValue); + } else { + objects.add(field.get(options)); + } + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + return objects.build(); + } +} http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/config/converters/ClassConverter.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/config/converters/ClassConverter.java b/src/main/java/org/apache/aurora/scheduler/config/converters/ClassConverter.java new file mode 100644 index 0000000..b2ecfa0 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/config/converters/ClassConverter.java @@ -0,0 +1,49 @@ +/** + * Licensed 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.aurora.scheduler.config.converters; + +import java.util.Map; + +import com.beust.jcommander.ParameterException; +import com.beust.jcommander.converters.BaseConverter; +import com.google.common.collect.ImmutableMap; + +import org.apache.aurora.scheduler.http.api.security.IniShiroRealmModule; +import org.apache.aurora.scheduler.http.api.security.Kerberos5ShiroRealmModule; + +public class ClassConverter extends BaseConverter> { + + private static final Map NAME_ALIASES = ImmutableMap.of( + "KERBEROS5_AUTHN", Kerberos5ShiroRealmModule.class.getCanonicalName(), + "INI_AUTHNZ", IniShiroRealmModule.class.getCanonicalName()); + + public ClassConverter(String optionName) { + super(optionName); + } + + @Override + public Class convert(String value) { + if (value.isEmpty()) { + throw new ParameterException(getErrorString(value, "must not be blank")); + } + + String unaliased = NAME_ALIASES.getOrDefault(value, value); + try { + return Class.forName(unaliased); + } catch (ClassNotFoundException e) { + throw new ParameterException(e); + } + } +} http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/config/converters/DataAmountConverter.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/config/converters/DataAmountConverter.java b/src/main/java/org/apache/aurora/scheduler/config/converters/DataAmountConverter.java new file mode 100644 index 0000000..a0d2620 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/config/converters/DataAmountConverter.java @@ -0,0 +1,57 @@ +/** + * Licensed 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.aurora.scheduler.config.converters; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import com.beust.jcommander.ParameterException; +import com.beust.jcommander.converters.BaseConverter; +import com.google.common.base.Functions; +import com.google.common.base.Optional; +import com.google.common.base.Predicates; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; + +import org.apache.aurora.common.quantity.Data; +import org.apache.aurora.scheduler.config.types.DataAmount; + +public class DataAmountConverter extends BaseConverter { + private static final Pattern AMOUNT_PATTERN = Pattern.compile("(\\d+)([A-Za-z]+)"); + + public DataAmountConverter(String optionName) { + super(optionName); + } + + @Override + public DataAmount convert(String raw) { + Matcher matcher = AMOUNT_PATTERN.matcher(raw); + + if (!matcher.matches()) { + throw new ParameterException(getErrorString(raw, "must be of the format 1KB, 2GB, etc.")); + } + + Optional unit = FluentIterable.from(Data.values()) + .firstMatch(Predicates.compose( + Predicates.equalTo(matcher.group(2)), + Functions.toStringFunction())); + if (unit.isPresent()) { + return new DataAmount(Integer.parseInt(matcher.group(1)), unit.get()); + } else { + throw new ParameterException( + getErrorString(raw, "one of " + ImmutableList.copyOf(Data.values()))); + } + } +} http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/config/converters/DockerParameterConverter.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/config/converters/DockerParameterConverter.java b/src/main/java/org/apache/aurora/scheduler/config/converters/DockerParameterConverter.java new file mode 100644 index 0000000..5c126e3 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/config/converters/DockerParameterConverter.java @@ -0,0 +1,36 @@ +/** + * Licensed 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.aurora.scheduler.config.converters; + +import com.beust.jcommander.ParameterException; +import com.beust.jcommander.converters.BaseConverter; + +import org.apache.aurora.gen.DockerParameter; + +public class DockerParameterConverter extends BaseConverter { + public DockerParameterConverter(String optionName) { + super(optionName); + } + + @Override + public DockerParameter convert(String value) { + int pos = value.indexOf('='); + if (pos == -1 || pos == 0 || pos == value.length() - 1) { + throw new ParameterException(getErrorString(value, "formatted as name=value")); + } + + return new DockerParameter(value.substring(0, pos), value.substring(pos + 1)); + } +} http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/config/converters/InetSocketAddressConverter.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/config/converters/InetSocketAddressConverter.java b/src/main/java/org/apache/aurora/scheduler/config/converters/InetSocketAddressConverter.java new file mode 100644 index 0000000..1261592 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/config/converters/InetSocketAddressConverter.java @@ -0,0 +1,32 @@ +/** + * Licensed 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.aurora.scheduler.config.converters; + +import java.net.InetSocketAddress; + +import com.beust.jcommander.converters.BaseConverter; + +import org.apache.aurora.common.net.InetSocketAddressHelper; + +public class InetSocketAddressConverter extends BaseConverter { + public InetSocketAddressConverter(String optionName) { + super(optionName); + } + + @Override + public InetSocketAddress convert(String value) { + return InetSocketAddressHelper.parse(value); + } +} http://git-wip-us.apache.org/repos/asf/aurora/blob/519e3df7/src/main/java/org/apache/aurora/scheduler/config/converters/TimeAmountConverter.java ---------------------------------------------------------------------- diff --git a/src/main/java/org/apache/aurora/scheduler/config/converters/TimeAmountConverter.java b/src/main/java/org/apache/aurora/scheduler/config/converters/TimeAmountConverter.java new file mode 100644 index 0000000..7e66cf2 --- /dev/null +++ b/src/main/java/org/apache/aurora/scheduler/config/converters/TimeAmountConverter.java @@ -0,0 +1,57 @@ +/** + * Licensed 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.aurora.scheduler.config.converters; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import com.beust.jcommander.ParameterException; +import com.beust.jcommander.converters.BaseConverter; +import com.google.common.base.Functions; +import com.google.common.base.Optional; +import com.google.common.base.Predicates; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; + +import org.apache.aurora.common.quantity.Time; +import org.apache.aurora.scheduler.config.types.TimeAmount; + +public class TimeAmountConverter extends BaseConverter { + private static final Pattern AMOUNT_PATTERN = Pattern.compile("(\\d+)([A-Za-z]+)"); + + public TimeAmountConverter(String optionName) { + super(optionName); + } + + @Override + public TimeAmount convert(String raw) { + Matcher matcher = AMOUNT_PATTERN.matcher(raw); + + if (!matcher.matches()) { + throw new ParameterException(getErrorString(raw, "must be of the format 1ns, 2secs, etc.")); + } + + Optional