[GitHub] StephanErb commented on a change in pull request #37: Staggered (Variable batch) Updates

2018-09-17 Thread GitBox
StephanErb commented on a change in pull request #37: Staggered (Variable 
batch) Updates
URL: https://github.com/apache/aurora/pull/37#discussion_r217963287
 
 

 ##
 File path: 
src/main/java/org/apache/aurora/scheduler/updater/strategy/VariableBatchStrategy.java
 ##
 @@ -0,0 +1,135 @@
+/**
+ * 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.updater.strategy;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Ordering;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A strategy that limits the number of instances selected by the subclass 
using variable steps.
+ *
+ * @param  Instance type.
+ */
+public class VariableBatchStrategy> implements 
UpdateStrategy {
+  private final Ordering ordering;
+  protected final ImmutableList maxActiveGroups;
+  private final boolean rollingForward;
+  private Optional instanceModificationCount;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(VariableBatchStrategy.class);
+
+  /**
+   * Creates a variable active-limited strategy that applies an upper bound to 
all results.
+   *
+   * @param maxActiveGroups  List of Maximum group sizes. Each group size 
represents a step.
+   * {@link #getNextGroup(Set, Set)}.
+   */
+  public VariableBatchStrategy(Ordering ordering,
+  List maxActiveGroups,
+  boolean rollingForward) {
+
+this.ordering = Objects.requireNonNull(ordering);
+this.rollingForward = rollingForward;
+
+maxActiveGroups.forEach(x -> Preconditions.checkArgument(x > 0));
+
+this.maxActiveGroups = ImmutableList.copyOf(maxActiveGroups);
+this.instanceModificationCount = Optional.empty();
+  }
+
+  private int determineStep(int idle) {
 
 Review comment:
   Please add one short comment that explains what this function does. For a 
casual reader it can be confusing what you mean with step.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] StephanErb commented on a change in pull request #37: Staggered (Variable batch) Updates

2018-09-17 Thread GitBox
StephanErb commented on a change in pull request #37: Staggered (Variable 
batch) Updates
URL: https://github.com/apache/aurora/pull/37#discussion_r217483009
 
 

 ##
 File path: docs/reference/configuration.md
 ##
 @@ -376,8 +376,34 @@ Parameters for controlling the rate and policy of rolling 
updates.
 | ```rollback_on_failure```| boolean  | When False, prevents auto rollback 
of a failed update (Default: True)
 | ```wait_for_batch_completion```| boolean | When True, all threads from a 
given batch will be blocked from picking up new instances until the entire 
batch is updated. This essentially simulates the legacy sequential updater 
algorithm. (Default: False)
 | ```pulse_interval_secs```| Integer  |  Indicates a [coordinated 
update](../features/job-updates.md#coordinated-job-updates). If no pulses are 
received within the provided interval the update will be blocked. Beta-updater 
only. Will fail on submission when used with client updater. (Default: None)
+| ```update_strategy```| Choice of ```QueueUpdateStrategy```,  
```BatchUpdateStrategy```, or ```VariableBatchUpdateStrategy``` object | 
Indicate which update strategy to use for this update.
 | ```sla_aware```  | boolean  | When True, updates will only 
update an instance if it does not break the task's specified [SLA 
Requirements](../features/sla-requirements.md). (Default: None)
 
+### QueueUpdateStrategy Objects
+
+Update strategy which will keep the active updating instances at size 
```batch_size``` throughout the update until there are no more instances left 
to update.
+
+| object   | type | description
+|  | :--: | 
+| ```batch_size``` | Integer  | Maximum number of shards to be 
updated in one iteration (Default: 1)
+
+### BatchUpdateStrategy Objects
+
+Update strategy which will wait until a maximum of ``batch_size`` number of 
instances are updated before continuing on to the next group until all 
instances are updated.
+
+| object   | type | description
+|  | :--: | 
+| ```batch_size``` | Integer  | Maximum number of shards to be 
updated in one iteration (Default: 1)
+
+### VariableBatchUpdateStrategy Objects
+
+Similar to Batch Update strategy, this strategy will wait until all instances 
in a current group are updated before updating more instances.
+However, with this strategy the size of each group may change as the update 
progresses.
 
 Review comment:
   This sentence seems to contradict `"Each batch behaves in the same way as 
the static batch update strategy where the group size remains constant 
throughout the update."` from the RELEASE-NOTES.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] StephanErb commented on a change in pull request #37: Staggered (Variable batch) Updates

2018-09-17 Thread GitBox
StephanErb commented on a change in pull request #37: Staggered (Variable 
batch) Updates
URL: https://github.com/apache/aurora/pull/37#discussion_r217481844
 
 

 ##
 File path: RELEASE-NOTES.md
 ##
 @@ -35,9 +35,23 @@
 retrying an SLA-aware kill (using a truncated binary backoff).
 3. `sla_aware_kill_retry_max_delay (default: 5mins)`: the maximum amount 
of time to wait before
 retrying an SLA-aware kill (using a truncated binary backoff).
+- New update strategy added: Variable Batch Update. With this strategy, a job 
may be updated in
 
 Review comment:
   Needs to be moved to a new release section, now that 0.21 is out.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] StephanErb commented on a change in pull request #37: Staggered (Variable batch) Updates

2018-09-17 Thread GitBox
StephanErb commented on a change in pull request #37: Staggered (Variable 
batch) Updates
URL: https://github.com/apache/aurora/pull/37#discussion_r217483391
 
 

 ##
 File path: examples/jobs/hello_world_variable_update.aurora
 ##
 @@ -0,0 +1,30 @@
+#
+# 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.
+#
+
+hello = Process(
+  name = 'hello',
+  cmdline = """
+while true; do
+  echo hello world
+  sleep 10
+done
+  """)
+
+task = SequentialTask(
+  processes = [hello],
+  resources = Resources(cpu = .10, ram = 8*MB, disk = 8*MB))
+
+jobs = [Service(
+  task = task, cluster = 'devcluster', role = 'www-data', environment = 
'prod', name = 'hello', instances = 6,
+update_config = UpdateConfig(update_strategy = 
VariableBatchUpdateStrategy(batch_sizes = [1,2,3])))]
 
 Review comment:
   Nitpick (as this is in our examples): Please format with multiple lines and 
indentation rather than having everything on one line.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] StephanErb commented on a change in pull request #37: Staggered (Variable batch) Updates

2018-09-17 Thread GitBox
StephanErb commented on a change in pull request #37: Staggered (Variable 
batch) Updates
URL: https://github.com/apache/aurora/pull/37#discussion_r217960469
 
 

 ##
 File path: 
src/main/java/org/apache/aurora/scheduler/storage/durability/ThriftBackfill.java
 ##
 @@ -143,8 +148,28 @@ public IJobUpdate backFillJobUpdate(JobUpdate update) {
   backfillTask(instructions.getDesiredState().getTask());
 }
 
+backfillUpdateStrategy(instructions.getSettings());
+
 instructions.getInitialState().forEach(e -> backfillTask(e.getTask()));
 
 return IJobUpdate.build(update);
   }
+
+  private void backfillUpdateStrategy(JobUpdateSettings settings) {
 
 Review comment:
   I think we should also perform the reverse operation here if possible: If 
only the new settings are set, also write the deprecated ones.
   
   This does not add much complexity but makes scheduler upgrade/downgrade 
paths easier to reason about.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] StephanErb commented on a change in pull request #37: Staggered (Variable batch) Updates

2018-09-17 Thread GitBox
StephanErb commented on a change in pull request #37: Staggered (Variable 
batch) Updates
URL: https://github.com/apache/aurora/pull/37#discussion_r217961218
 
 

 ##
 File path: 
src/main/java/org/apache/aurora/scheduler/updater/strategy/VariableBatchStrategy.java
 ##
 @@ -0,0 +1,135 @@
+/**
+ * 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.updater.strategy;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Ordering;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A strategy that limits the number of instances selected by the subclass 
using variable steps.
+ *
+ * @param  Instance type.
+ */
+public class VariableBatchStrategy> implements 
UpdateStrategy {
+  private final Ordering ordering;
+  protected final ImmutableList maxActiveGroups;
+  private final boolean rollingForward;
+  private Optional instanceModificationCount;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(VariableBatchStrategy.class);
+
+  /**
+   * Creates a variable active-limited strategy that applies an upper bound to 
all results.
+   *
+   * @param maxActiveGroups  List of Maximum group sizes. Each group size 
represents a step.
+   * {@link #getNextGroup(Set, Set)}.
+   */
+  public VariableBatchStrategy(Ordering ordering,
+  List maxActiveGroups,
+  boolean rollingForward) {
+
+this.ordering = Objects.requireNonNull(ordering);
+this.rollingForward = rollingForward;
+
+maxActiveGroups.forEach(x -> Preconditions.checkArgument(x > 0));
+
+this.maxActiveGroups = ImmutableList.copyOf(maxActiveGroups);
+this.instanceModificationCount = Optional.empty();
+  }
+
+  private int determineStep(int idle) {
+
+// Calculate which step we are in by finding out how many instances we 
have left to update.
+int scheduled = instanceModificationCount.get() - idle;
+
+int step = 0;
+int sum = 0;
+
+LOG.info("Update progress {} changed, {} idle, and {} total to be 
changed.",
+scheduled,
+idle,
+instanceModificationCount.get());
+
+if (rollingForward) {
+  while (sum < scheduled && step < maxActiveGroups.size()) {
+sum += maxActiveGroups.get(step);
+
+++step;
+  }
+} else {
+
+  // Starting with the first step as we're now comparing to idle instead 
of scheduled
+  // to work backwards from the last step that was executed in the batch 
steps.
+  sum = maxActiveGroups.get(step);
+
+  // TODO(rdelvalle): Consider if it's necessary to handle fractional 
steps.
+  // (i.e. halfway between step X and X+1)
 
 Review comment:
   I think now would be the best time to reason about this. I don't think 
fractional steps are necessary, but maybe you can elaborate what you had in 
mind.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services