You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by se...@apache.org on 2017/03/13 21:22:06 UTC

aurora git commit: Change Resource Validation in ConfigurationManager so that it validates the Resource Set instead of deprecated fields

Repository: aurora
Updated Branches:
  refs/heads/master a07b9edf5 -> 2f08d9110


Change Resource Validation in ConfigurationManager so that it validates the Resource Set instead of deprecated fields

The Resource validation in ConfigurationManager is now done against the Resource set instead of the NumCpus, RamMb and DiskMb fields.

Related Issue: AURORA-1707

Reviewed at https://reviews.apache.org/r/56395/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/2f08d911
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/2f08d911
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/2f08d911

Branch: refs/heads/master
Commit: 2f08d9110eb180968b2633523f1c2386874790e9
Parents: a07b9ed
Author: Nicol�s Donatucci <nd...@medallia.com>
Authored: Mon Mar 13 21:59:42 2017 +0100
Committer: Stephan Erb <se...@apache.org>
Committed: Mon Mar 13 21:59:42 2017 +0100

----------------------------------------------------------------------
 .../configuration/ConfigurationManager.java     | 44 ++++++--------------
 .../scheduler/storage/log/ThriftBackfill.java   | 16 ++++---
 src/main/python/apache/aurora/config/thrift.py  |  2 +-
 .../configuration/ConfigurationManagerTest.java | 11 -----
 .../thrift/SchedulerThriftInterfaceTest.java    |  4 ++
 5 files changed, 28 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/2f08d911/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
index 80f0aeb..ad6b3ef 100644
--- a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
+++ b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
@@ -23,7 +23,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Predicate;
 import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
@@ -33,7 +32,6 @@ import org.apache.aurora.gen.Container;
 import org.apache.aurora.gen.DockerParameter;
 import org.apache.aurora.gen.JobConfiguration;
 import org.apache.aurora.gen.TaskConfig;
-import org.apache.aurora.gen.TaskConfig._Fields;
 import org.apache.aurora.gen.TaskConstraint;
 import org.apache.aurora.scheduler.TierManager;
 import org.apache.aurora.scheduler.base.JobKeys;
@@ -53,8 +51,11 @@ import org.apache.aurora.scheduler.storage.log.ThriftBackfill;
 
 import static java.util.Objects.requireNonNull;
 
+import static org.apache.aurora.scheduler.resources.ResourceType.CPUS;
+import static org.apache.aurora.scheduler.resources.ResourceType.DISK_MB;
 import static org.apache.aurora.scheduler.resources.ResourceType.GPUS;
 import static org.apache.aurora.scheduler.resources.ResourceType.PORTS;
+import static org.apache.aurora.scheduler.resources.ResourceType.RAM_MB;
 
 /**
  * Manages translation from a string-mapped configuration to a concrete configuration type, and
@@ -87,31 +88,6 @@ public class ConfigurationManager {
     }
   }
 
-  private static class RequiredFieldValidator<T> implements Validator<TaskConfig> {
-    private final _Fields field;
-    private final Validator<T> validator;
-
-    RequiredFieldValidator(_Fields field, Validator<T> validator) {
-      this.field = field;
-      this.validator = validator;
-    }
-
-    public void validate(TaskConfig task) throws TaskDescriptionException {
-      if (!task.isSet(field)) {
-        throw new TaskDescriptionException("Field " + field.getFieldName() + " is required.");
-      }
-      @SuppressWarnings("unchecked")
-      T value = (T) task.getFieldValue(field);
-      validator.validate(value);
-    }
-  }
-
-  private static final Iterable<RequiredFieldValidator<?>> REQUIRED_FIELDS_VALIDATORS =
-      ImmutableList.of(
-          new RequiredFieldValidator<>(_Fields.NUM_CPUS, new GreaterThan(0.0, "num_cpus")),
-          new RequiredFieldValidator<>(_Fields.RAM_MB, new GreaterThan(0.0, "ram_mb")),
-          new RequiredFieldValidator<>(_Fields.DISK_MB, new GreaterThan(0.0, "disk_mb")));
-
   public static class ConfigurationManagerSettings {
     private final ImmutableSet<Container._Fields> allowedContainerTypes;
     private final boolean allowDockerParameters;
@@ -352,10 +328,6 @@ public class ConfigurationManager {
 
     thriftBackfill.backfillTask(builder);
 
-    for (RequiredFieldValidator<?> validator : REQUIRED_FIELDS_VALIDATORS) {
-      validator.validate(builder);
-    }
-
     String types = config.getResources().stream()
         .collect(Collectors.groupingBy(e -> ResourceType.fromResource(e)))
         .entrySet().stream()
@@ -368,6 +340,16 @@ public class ConfigurationManager {
       throw new TaskDescriptionException("Multiple resource values are not supported for " + types);
     }
 
+    Validator<Number> cpuvalidator = new GreaterThan(0.0, "num_cpus");
+    cpuvalidator.validate(
+            ResourceManager.quantityOf(ResourceManager.getTaskResources(config, CPUS)));
+    Validator<Number> ramvalidator = new GreaterThan(0.0, "ram_mb");
+    ramvalidator.validate(
+            ResourceManager.quantityOf(ResourceManager.getTaskResources(config, RAM_MB)));
+    Validator<Number> diskvalidator = new GreaterThan(0.0, "disk_mb");
+    diskvalidator.validate(
+            ResourceManager.quantityOf(ResourceManager.getTaskResources(config, DISK_MB)));
+
     if (!settings.allowGpuResource && config.getResources().stream()
         .filter(r -> ResourceType.fromResource(r).equals(GPUS))
         .findAny()

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f08d911/src/main/java/org/apache/aurora/scheduler/storage/log/ThriftBackfill.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/log/ThriftBackfill.java b/src/main/java/org/apache/aurora/scheduler/storage/log/ThriftBackfill.java
index c883843..3567cf0 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/log/ThriftBackfill.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/log/ThriftBackfill.java
@@ -60,9 +60,10 @@ public final class ThriftBackfill {
 
   private static Resource getResource(Set<Resource> resources, ResourceType type) {
     return resources.stream()
-        .filter(e -> ResourceType.fromResource(IResource.build(e)).equals(type))
-        .findFirst()
-        .orElseThrow(() -> new IllegalArgumentException("Missing resource definition for " + type));
+            .filter(e -> ResourceType.fromResource(IResource.build(e)).equals(type))
+            .findFirst()
+            .orElseThrow(() ->
+                    new IllegalArgumentException("Missing resource definition for " + type));
   }
 
   /**
@@ -162,9 +163,12 @@ public final class ThriftBackfill {
 
         throw new IllegalArgumentException("Quota resources must be exactly: " + quotaResources);
       }
-      aggregate.setNumCpus(getResource(aggregate.getResources(), CPUS).getNumCpus());
-      aggregate.setRamMb(getResource(aggregate.getResources(), RAM_MB).getRamMb());
-      aggregate.setDiskMb(getResource(aggregate.getResources(), DISK_MB).getDiskMb());
+      aggregate.setNumCpus(
+              getResource(aggregate.getResources(), CPUS).getNumCpus());
+      aggregate.setRamMb(
+              getResource(aggregate.getResources(), RAM_MB).getRamMb());
+      aggregate.setDiskMb(
+              getResource(aggregate.getResources(), DISK_MB).getDiskMb());
     }
     return IResourceAggregate.build(aggregate);
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f08d911/src/main/python/apache/aurora/config/thrift.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/config/thrift.py b/src/main/python/apache/aurora/config/thrift.py
index 2158c3a..601e1fc 100644
--- a/src/main/python/apache/aurora/config/thrift.py
+++ b/src/main/python/apache/aurora/config/thrift.py
@@ -294,7 +294,7 @@ def convert(job, metadata=frozenset(), ports=frozenset()):
        Resource(ramMb=task.ramMb),
        Resource(diskMb=task.diskMb)]
       + [Resource(namedPort=p) for p in ports]
-      + [Resource(numGpus=numGpus)] if numGpus else [])
+      + ([Resource(numGpus=numGpus)] if numGpus else []))
 
   task.job = key
   task.owner = owner

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f08d911/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java b/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
index 5419d7e..d6904f8 100644
--- a/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/configuration/ConfigurationManagerTest.java
@@ -263,17 +263,6 @@ public class ConfigurationManagerTest {
   }
 
   @Test
-  public void testTaskResourceBackfill() throws Exception {
-    TaskConfig builder = CONFIG_WITH_CONTAINER.newBuilder();
-    builder.unsetResources();
-
-    assertFalse(builder.isSetResources());
-    ITaskConfig populated =
-        DOCKER_CONFIGURATION_MANAGER.validateAndPopulate(ITaskConfig.build(builder));
-    assertEquals(CONFIG_WITH_CONTAINER.getResources(), populated.getResources());
-  }
-
-  @Test
   public void testMultipleResourceValuesBlocked() throws Exception {
     TaskConfig builder = CONFIG_WITH_CONTAINER.newBuilder();
     builder.addToResources(numCpus(3.0));

http://git-wip-us.apache.org/repos/asf/aurora/blob/2f08d911/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
index c36abc8..0ff6ce0 100644
--- a/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/thrift/SchedulerThriftInterfaceTest.java
@@ -598,6 +598,10 @@ public class SchedulerThriftInterfaceTest extends EasyMockTest {
         .setNumCpus(1.0)
         .setRamMb(1024)
         .setDiskMb(1024)
+        .setResources(ImmutableSet.of(
+            numCpus(1.0),
+            ramMb(1024),
+            diskMb(1024)))
         .setIsService(true)
         .setProduction(true)
         .setTier(TaskTestUtil.PROD_TIER_NAME)