[3/8] aurora git commit: Break apart async package and AsyncModule into purpose-specific equivalents.

2015-07-22 Thread wfarner
http://git-wip-us.apache.org/repos/asf/aurora/blob/0070a5fd/src/test/java/org/apache/aurora/scheduler/async/TaskTimeoutTest.java
--
diff --git 
a/src/test/java/org/apache/aurora/scheduler/async/TaskTimeoutTest.java 
b/src/test/java/org/apache/aurora/scheduler/async/TaskTimeoutTest.java
deleted file mode 100644
index b98a8d7..000
--- a/src/test/java/org/apache/aurora/scheduler/async/TaskTimeoutTest.java
+++ /dev/null
@@ -1,244 +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.async;
-
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableList;
-import com.twitter.common.quantity.Amount;
-import com.twitter.common.quantity.Time;
-import com.twitter.common.stats.StatsProvider;
-import com.twitter.common.testing.easymock.EasyMockTest;
-import com.twitter.common.util.testing.FakeClock;
-
-import org.apache.aurora.gen.AssignedTask;
-import org.apache.aurora.gen.ScheduleStatus;
-import org.apache.aurora.gen.ScheduledTask;
-import org.apache.aurora.gen.TaskConfig;
-import org.apache.aurora.gen.TaskEvent;
-import org.apache.aurora.scheduler.events.PubsubEvent.TaskStateChange;
-import org.apache.aurora.scheduler.state.StateChangeResult;
-import org.apache.aurora.scheduler.state.StateManager;
-import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
-import org.apache.aurora.scheduler.storage.testing.StorageTestUtil;
-import org.easymock.Capture;
-import org.easymock.EasyMock;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.apache.aurora.gen.ScheduleStatus.ASSIGNED;
-import static org.apache.aurora.gen.ScheduleStatus.FINISHED;
-import static org.apache.aurora.gen.ScheduleStatus.INIT;
-import static org.apache.aurora.gen.ScheduleStatus.KILLED;
-import static org.apache.aurora.gen.ScheduleStatus.KILLING;
-import static org.apache.aurora.gen.ScheduleStatus.LOST;
-import static org.apache.aurora.gen.ScheduleStatus.PENDING;
-import static org.apache.aurora.gen.ScheduleStatus.PREEMPTING;
-import static org.apache.aurora.gen.ScheduleStatus.RUNNING;
-import static org.apache.aurora.gen.ScheduleStatus.STARTING;
-import static org.easymock.EasyMock.eq;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.junit.Assert.assertEquals;
-
-public class TaskTimeoutTest extends EasyMockTest {
-
-  private static final String TASK_ID = task_id;
-  private static final AmountLong, Time TIMEOUT = Amount.of(1L, 
Time.MINUTES);
-
-  private AtomicLong timedOutTaskCounter;
-  private ScheduledExecutorService executor;
-  private StorageTestUtil storageUtil;
-  private ScheduledFuture? future;
-  private StateManager stateManager;
-  private FakeClock clock;
-  private TaskTimeout timeout;
-  private StatsProvider statsProvider;
-
-  @Before
-  public void setUp() {
-executor = createMock(ScheduledExecutorService.class);
-storageUtil = new StorageTestUtil(this);
-storageUtil.expectOperations();
-future = createMock(new ClazzScheduledFuture?() { });
-stateManager = createMock(StateManager.class);
-clock = new FakeClock();
-statsProvider = createMock(StatsProvider.class);
-timedOutTaskCounter = new AtomicLong();
-expect(statsProvider.makeCounter(TaskTimeout.TIMED_OUT_TASKS_COUNTER))
-.andReturn(timedOutTaskCounter);
-  }
-
-  private void replayAndCreate() {
-control.replay();
-timeout = new TaskTimeout(
-executor,
-storageUtil.storage,
-stateManager,
-TIMEOUT,
-statsProvider);
-timeout.startAsync().awaitRunning();
-  }
-
-  private CaptureRunnable expectTaskWatch(AmountLong, Time expireIn) {
-CaptureRunnable capture = createCapture();
-executor.schedule(
-EasyMock.capture(capture),
-eq((long) expireIn.getValue()),
-eq(expireIn.getUnit().getTimeUnit()));
-expectLastCall().andReturn(future);
-return capture;
-  }
-
-  private CaptureRunnable expectTaskWatch() {
-return expectTaskWatch(TIMEOUT);
-  }
-
-  private void changeState(String taskId, ScheduleStatus from, ScheduleStatus 
to) {
-IScheduledTask task = IScheduledTask.build(new ScheduledTask()
-.setStatus(to)
-.setAssignedTask(new 

[6/8] aurora git commit: Break apart async package and AsyncModule into purpose-specific equivalents.

2015-07-22 Thread wfarner
http://git-wip-us.apache.org/repos/asf/aurora/blob/0070a5fd/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java
--
diff --git a/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java 
b/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java
new file mode 100644
index 000..14bf265
--- /dev/null
+++ b/src/main/java/org/apache/aurora/scheduler/offers/OfferManager.java
@@ -0,0 +1,408 @@
+/**
+ * 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.offers;
+
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.inject.Inject;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import com.google.common.base.Supplier;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Ordering;
+import com.google.common.eventbus.Subscribe;
+import com.twitter.common.inject.TimedInterceptor.Timed;
+import com.twitter.common.quantity.Amount;
+import com.twitter.common.quantity.Time;
+import com.twitter.common.stats.Stats;
+
+import org.apache.aurora.gen.MaintenanceMode;
+import org.apache.aurora.scheduler.HostOffer;
+import org.apache.aurora.scheduler.async.AsyncModule.AsyncExecutor;
+import org.apache.aurora.scheduler.base.TaskGroupKey;
+import org.apache.aurora.scheduler.events.PubsubEvent.DriverDisconnected;
+import org.apache.aurora.scheduler.events.PubsubEvent.EventSubscriber;
+import org.apache.aurora.scheduler.mesos.Driver;
+import org.apache.aurora.scheduler.state.TaskAssigner.Assignment;
+import org.apache.aurora.scheduler.storage.entities.IHostAttributes;
+import org.apache.mesos.Protos.OfferID;
+import org.apache.mesos.Protos.SlaveID;
+
+import static java.util.Objects.requireNonNull;
+
+import static org.apache.aurora.gen.MaintenanceMode.DRAINED;
+import static org.apache.aurora.gen.MaintenanceMode.DRAINING;
+import static org.apache.aurora.gen.MaintenanceMode.NONE;
+import static org.apache.aurora.gen.MaintenanceMode.SCHEDULED;
+import static 
org.apache.aurora.scheduler.events.PubsubEvent.HostAttributesChanged;
+
+/**
+ * Tracks the Offers currently known by the scheduler.
+ */
+public interface OfferManager extends EventSubscriber {
+
+  /**
+   * Notifies the scheduler of a new resource offer.
+   *
+   * @param offer Newly-available resource offer.
+   */
+  void addOffer(HostOffer offer);
+
+  /**
+   * Invalidates an offer.  This indicates that the scheduler should not 
attempt to match any
+   * tasks against the offer.
+   *
+   * @param offer Canceled offer.
+   */
+  void cancelOffer(OfferID offer);
+
+  /**
+   * Launches the first task that satisfies the {@code acceptor} by returning 
a {@link Assignment}.
+   *
+   * @param acceptor Function that determines if an offer is accepted.
+   * @param groupKey Task group key.
+   * @return {@code true} if the task was launched, {@code false} if no offers 
satisfied the
+   * {@code acceptor}.
+   * @throws LaunchException If the acceptor accepted an offer, but there was 
an error launching the
+   * task.
+   */
+  boolean launchFirst(FunctionHostOffer, Assignment acceptor, TaskGroupKey 
groupKey)
+  throws LaunchException;
+
+  /**
+   * Notifies the offer queue that a host's attributes have changed.
+   *
+   * @param change State change notification.
+   */
+  void hostAttributesChanged(HostAttributesChanged change);
+
+  /**
+   * Gets the offers that the scheduler is holding.
+   *
+   * @return A snapshot of the offers that the scheduler is currently holding.
+   */
+  IterableHostOffer getOffers();
+
+  /**
+   * Gets an offer for the given slave ID.
+   *
+   * @param slaveId Slave ID to get offer for.
+   * @return An offer for the slave ID.
+   */
+  OptionalHostOffer getOffer(SlaveID slaveId);
+
+  /**
+   * Calculates the amount of time before an offer should be 'returned' by 
declining it.
+   * The delay is calculated for each 

[7/8] aurora git commit: Break apart async package and AsyncModule into purpose-specific equivalents.

2015-07-22 Thread wfarner
http://git-wip-us.apache.org/repos/asf/aurora/blob/0070a5fd/src/main/java/org/apache/aurora/scheduler/async/TaskReconciler.java
--
diff --git 
a/src/main/java/org/apache/aurora/scheduler/async/TaskReconciler.java 
b/src/main/java/org/apache/aurora/scheduler/async/TaskReconciler.java
deleted file mode 100644
index 68d2e77..000
--- a/src/main/java/org/apache/aurora/scheduler/async/TaskReconciler.java
+++ /dev/null
@@ -1,155 +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.async;
-
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.atomic.AtomicLong;
-
-import javax.inject.Inject;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.util.concurrent.AbstractIdleService;
-import com.twitter.common.quantity.Amount;
-import com.twitter.common.quantity.Time;
-import com.twitter.common.stats.StatsProvider;
-
-import org.apache.aurora.scheduler.base.Query;
-import org.apache.aurora.scheduler.base.Tasks;
-import org.apache.aurora.scheduler.mesos.Driver;
-import org.apache.aurora.scheduler.storage.Storage;
-import org.apache.aurora.scheduler.storage.entities.IScheduledTask;
-import org.apache.mesos.Protos;
-
-import static java.util.Objects.requireNonNull;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.twitter.common.quantity.Time.MINUTES;
-
-/**
- * A task reconciler that periodically triggers Mesos (implicit) and Aurora 
(explicit) task
- * reconciliation to synchronize global task states. More on task 
reconciliation:
- * http://mesos.apache.org/documentation/latest/reconciliation.
- */
-public class TaskReconciler extends AbstractIdleService {
-
-  @VisibleForTesting
-  static final String EXPLICIT_STAT_NAME = reconciliation_explicit_runs;
-
-  @VisibleForTesting
-  static final String IMPLICIT_STAT_NAME = reconciliation_implicit_runs;
-
-  private final TaskReconcilerSettings settings;
-  private final Storage storage;
-  private final Driver driver;
-  private final ScheduledExecutorService executor;
-  private final AtomicLong explicitRuns;
-  private final AtomicLong implicitRuns;
-
-  static class TaskReconcilerSettings {
-private final AmountLong, Time explicitInterval;
-private final AmountLong, Time implicitInterval;
-private final long explicitDelayMinutes;
-private final long implicitDelayMinutes;
-
-@VisibleForTesting
-TaskReconcilerSettings(
-AmountLong, Time initialDelay,
-AmountLong, Time explicitInterval,
-AmountLong, Time implicitInterval,
-AmountLong, Time scheduleSpread) {
-
-  this.explicitInterval = requireNonNull(explicitInterval);
-  this.implicitInterval = requireNonNull(implicitInterval);
-  explicitDelayMinutes = requireNonNull(initialDelay).as(MINUTES);
-  implicitDelayMinutes = initialDelay.as(MINUTES) + 
scheduleSpread.as(MINUTES);
-  checkArgument(
-  explicitDelayMinutes = 0,
-  Invalid explicit reconciliation delay:  + explicitDelayMinutes);
-  checkArgument(
-  implicitDelayMinutes = 0L,
-  Invalid implicit reconciliation delay:  + implicitDelayMinutes);
-}
-  }
-
-  @Inject
-  TaskReconciler(
-  TaskReconcilerSettings settings,
-  Storage storage,
-  Driver driver,
-  ScheduledExecutorService executor,
-  StatsProvider stats) {
-
-this.settings = requireNonNull(settings);
-this.storage = requireNonNull(storage);
-this.driver = requireNonNull(driver);
-this.executor = requireNonNull(executor);
-this.explicitRuns = stats.makeCounter(EXPLICIT_STAT_NAME);
-this.implicitRuns = stats.makeCounter(IMPLICIT_STAT_NAME);
-  }
-
-  @Override
-  protected void startUp() {
-// Schedule explicit reconciliation.
-executor.scheduleAtFixedRate(
-new Runnable() {
-  @Override
-  public void run() {
-ImmutableSetProtos.TaskStatus active = FluentIterable
-.from(Storage.Util.fetchTasks(
-storage,
-Query.unscoped().byStatus(Tasks.SLAVE_ASSIGNED_STATES)))
-.transform(TASK_TO_PROTO)
-.toSet();
-
-driver.reconcileTasks(active);
-

aurora git commit: Make Storage.Work an interface instead of an abstract class.

2015-07-22 Thread kevints
Repository: aurora
Updated Branches:
  refs/heads/master 38c2e769c - 96b56b885


Make Storage.Work an interface instead of an abstract class.

Make `Storage.Work` and friends interfaces instead of abstract classes.

This allows you to pass lambdas and method references to calls to
`storage.work`. The main ripple is that as interface methods, `apply`
and `execute` must be public. This was done programmatically with a sed
script.

Testing Done:
./gradlew -Pq build

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


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

Branch: refs/heads/master
Commit: 96b56b8851eb9c42b87a057afed1171610198b3f
Parents: 38c2e76
Author: Kevin Sweeney kevi...@apache.org
Authored: Wed Jul 22 22:44:00 2015 -0700
Committer: Kevin Sweeney kevi...@apache.org
Committed: Wed Jul 22 22:44:00 2015 -0700

--
 .../aurora/benchmark/SchedulingBenchmarks.java  |  4 ++--
 .../aurora/benchmark/StatusUpdateBenchmark.java |  2 +-
 .../aurora/benchmark/ThriftApiBenchmarks.java   |  2 +-
 .../aurora/benchmark/UpdateStoreBenchmarks.java |  4 ++--
 .../aurora/scheduler/SchedulerLifecycle.java|  9 +--
 .../aurora/scheduler/TaskStatusHandlerImpl.java |  2 +-
 .../scheduler/cron/quartz/AuroraCronJob.java|  4 ++--
 .../cron/quartz/CronJobManagerImpl.java |  2 +-
 .../scheduler/mesos/MesosSchedulerImpl.java |  4 ++--
 .../scheduler/pruning/TaskHistoryPruner.java|  2 +-
 .../scheduler/scheduling/TaskThrottler.java |  2 +-
 .../storage/CallOrderEnforcingStorage.java  |  2 +-
 .../aurora/scheduler/storage/Storage.java   | 25 +---
 .../scheduler/storage/backup/Recovery.java  |  2 +-
 .../storage/backup/TemporaryStorage.java|  2 +-
 .../storage/db/RowGarbageCollector.java |  2 +-
 .../scheduler/storage/log/LogStorage.java   |  8 +++
 .../storage/log/SnapshotStoreImpl.java  |  2 +-
 .../thrift/SchedulerThriftInterface.java|  8 +++
 .../updater/JobUpdateControllerImpl.java| 12 +-
 .../cron/quartz/AuroraCronJobTest.java  |  4 ++--
 .../cron/quartz/CronJobManagerImplTest.java |  2 +-
 .../scheduling/TaskSchedulerImplTest.java   |  2 +-
 .../scheduler/scheduling/TaskSchedulerTest.java |  6 ++---
 .../scheduler/state/StateManagerImplTest.java   | 10 
 .../scheduler/stats/ResourceCounterTest.java|  4 ++--
 .../storage/AbstractCronJobStoreTest.java   | 10 
 .../storage/AbstractTaskStoreTest.java  | 10 
 .../scheduler/storage/StorageBackfillTest.java  |  4 ++--
 .../storage/db/DbAttributeStoreTest.java|  6 ++---
 .../scheduler/storage/db/DbStorageTest.java |  2 +-
 .../scheduler/storage/log/LogStorageTest.java   |  8 +++
 .../storage/mem/InMemTaskStoreTest.java |  2 +-
 .../storage/mem/StorageTransactionTest.java | 22 -
 .../aurora/scheduler/updater/JobUpdaterIT.java  | 22 -
 35 files changed, 107 insertions(+), 107 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/aurora/blob/96b56b88/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java
--
diff --git a/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java 
b/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java
index 5716f23..5bc73d5 100644
--- a/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java
+++ b/src/jmh/java/org/apache/aurora/benchmark/SchedulingBenchmarks.java
@@ -206,7 +206,7 @@ public class SchedulingBenchmarks {
 private void saveTasks(final SetIScheduledTask tasks) {
   storage.write(new Storage.MutateWork.NoResult.Quiet() {
 @Override
-protected void execute(Storage.MutableStoreProvider storeProvider) {
+public void execute(Storage.MutableStoreProvider storeProvider) {
   storeProvider.getUnsafeTaskStore().saveTasks(tasks);
 }
   });
@@ -215,7 +215,7 @@ public class SchedulingBenchmarks {
 private void saveHostAttributes(final SetIHostAttributes 
hostAttributesToSave) {
   storage.write(new Storage.MutateWork.NoResult.Quiet() {
 @Override
-protected void execute(Storage.MutableStoreProvider storeProvider) {
+public void execute(Storage.MutableStoreProvider storeProvider) {
   for (IHostAttributes attributes : hostAttributesToSave) {
 storeProvider.getAttributeStore().saveHostAttributes(attributes);
   }

http://git-wip-us.apache.org/repos/asf/aurora/blob/96b56b88/src/jmh/java/org/apache/aurora/benchmark/StatusUpdateBenchmark.java

aurora git commit: Added fix for client to connect to correct zookeeper-hosts address/port pairs.

2015-07-22 Thread zmanji
Repository: aurora
Updated Branches:
  refs/heads/master 0070a5fd1 - 8bdfb8500


Added fix for client to connect to correct zookeeper-hosts address/port pairs.

When more than one zookeeper host was specified in the clusters.json file, only
the last host was accessed through the specified zk_port. The client would
attempt to connect to all other zookeeper hosts using the default port (2181).

For example, in clusters.json:
[{
  name: my_cluster,
  zk: host1.domain.com,host2.domain.com,host3.domain.com,
  zk_port: 2626,
  scheduler_zk_path: /cluster/aurora/scheduler,
  slave_run_directory: latest
}
...
]
I found that the client would attempt to access the zk hosts addresses
host1.domain.com:2181, host1.domain.com:2181, and host3.domain.com:2626. Thus,
the zk_port attribute would only update the last host in the host string-list.
This change allows the client to properly connect to multiple zookeeper hosts
using a port other than the specified port.

Note, that the KazooClient accepts a comma-separated string of hosts.

Testing Done:
This is a simple change, but tested to make sure client can still connect to
each zookeeper host. Then verified that all hosts are now accessed through the
correct port, not just the default (port 2181). Also viewable by Aurora client
INFO logs.

Bugs closed: AURORA-1405

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


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

Branch: refs/heads/master
Commit: 8bdfb8500e792da199bd8cc9fed38d36e2448e81
Parents: 0070a5f
Author: Zane Silver zsil...@luminatewireless.com
Authored: Wed Jul 22 14:26:57 2015 -0700
Committer: zma...@apache.org zma...@twitter.com
Committed: Wed Jul 22 14:26:57 2015 -0700

--
 src/main/python/apache/aurora/client/api/scheduler_client.py | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/aurora/blob/8bdfb850/src/main/python/apache/aurora/client/api/scheduler_client.py
--
diff --git a/src/main/python/apache/aurora/client/api/scheduler_client.py 
b/src/main/python/apache/aurora/client/api/scheduler_client.py
index b8361b8..8e91788 100644
--- a/src/main/python/apache/aurora/client/api/scheduler_client.py
+++ b/src/main/python/apache/aurora/client/api/scheduler_client.py
@@ -120,7 +120,8 @@ class ZookeeperSchedulerClient(SchedulerClient):
 if cluster.scheduler_zk_path is None:
   raise ValueError('Cluster has no defined scheduler path, must specify 
scheduler_zk_path '
'in your cluster config!')
-zk = TwitterKazooClient.make(str('%s:%s' % (cluster.zk, port)), 
verbose=verbose)
+hosts = [h + ':{p}' for h in cluster.zk.split(',')]
+zk = TwitterKazooClient.make(str(','.join(hosts).format(p=port)), 
verbose=verbose)
 return zk, ServerSet(zk, cluster.scheduler_zk_path, **kw)
 
   def __init__(self, cluster, port=2181, verbose=False, _deadline=deadline, 
**kwargs):



aurora git commit: Add support for arbitrary Docker parameters.

2015-07-22 Thread wfarner
Repository: aurora
Updated Branches:
  refs/heads/master 8bdfb8500 - b002e4223


Add support for arbitrary Docker parameters.

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


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

Branch: refs/heads/master
Commit: b002e42233b2ebc2618063b05a97b4f89da6baf2
Parents: 8bdfb85
Author: Mauricio Garavaglia mauriciogaravag...@gmail.com
Authored: Wed Jul 22 15:40:13 2015 -0700
Committer: Bill Farner wfar...@apache.org
Committed: Wed Jul 22 15:40:13 2015 -0700

--
 .../thrift/org/apache/aurora/gen/api.thrift | 10 +++
 docs/configuration-reference.md | 17 +--
 .../configuration/ConfigurationManager.java | 15 --
 .../scheduler/mesos/MesosTaskFactory.java   | 14 +++--
 .../python/apache/aurora/config/schema/base.py  |  4 +++
 src/main/python/apache/aurora/config/thrift.py  |  7 -
 .../mesos/MesosTaskFactoryImplTest.java | 31 +---
 .../python/apache/aurora/config/test_thrift.py  | 19 +++-
 8 files changed, 104 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/aurora/blob/b002e422/api/src/main/thrift/org/apache/aurora/gen/api.thrift
--
diff --git a/api/src/main/thrift/org/apache/aurora/gen/api.thrift 
b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
index d740a90..f792be0 100644
--- a/api/src/main/thrift/org/apache/aurora/gen/api.thrift
+++ b/api/src/main/thrift/org/apache/aurora/gen/api.thrift
@@ -208,10 +208,20 @@ struct Volume {
 struct MesosContainer {
 }
 
+/** Describes a parameter passed to docker cli */
+struct DockerParameter {
+  /** a parameter to pass to docker. (e.g. volume) */
+  1: string name
+  /** the value to pass to a parameter (e.g. /src/webapp:/opt/webapp) */
+  2: string value
+}
+
 /** Describes a docker container */
 struct DockerContainer {
   /** The container image to be run */
   1: string image
+  /** The arbitrary parameters to pass to container */
+  2: optional listDockerParameter parameters
 }
 
 /** Describes a container to be used in a task */

http://git-wip-us.apache.org/repos/asf/aurora/blob/b002e422/docs/configuration-reference.md
--
diff --git a/docs/configuration-reference.md b/docs/configuration-reference.md
index dafd306..ad2701c 100644
--- a/docs/configuration-reference.md
+++ b/docs/configuration-reference.md
@@ -417,9 +417,20 @@ Describes the container the job's processes will run 
inside.
 
 ### Docker Object
 
-  param  | type   | description
-  -  | :: | ---
-  ```image```| String | The name of the docker image to execute.  
If the image does not exist locally it will be pulled with ```docker pull```.
+  param| type| description
+  -| ::  | ---
+  ```image```  | String  | The name of the docker image to 
execute.  If the image does not exist locally it will be pulled with ```docker 
pull```.
+  ```parameters``` | List(Parameter) | Additional parameters to pass to the 
docker containerizer.
+
+### Docker Parameter Object
+
+Docker CLI parameters. This needs to be enabled by the scheduler 
`enable_docker_parameters` option.
+See [Docker Command Line 
Reference](https://docs.docker.com/reference/commandline/run/) for valid 
parameters. 
+
+  param| type| description
+  -| ::  | ---
+  ```name```   | String  | The name of the docker parameter. E.g. 
volume
+  ```value```  | String  | The value of the parameter. E.g. 
/usr/local/bin:/usr/bin:rw
 
 ### LifecycleConfig Objects
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/b002e422/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 be79e70..d103d19 100644
--- 
a/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
+++ 
b/src/main/java/org/apache/aurora/scheduler/configuration/ConfigurationManager.java
@@ -63,6 +63,10 @@ public final class ConfigurationManager {
   private static final ArgListContainer._Fields ALLOWED_CONTAINER_TYPES =
   Arg.create(ImmutableList.of(Container._Fields.MESOS));
 
+  @CmdLine(name = allow_docker_parameters,
+  help = 

aurora git commit: Replace Guava Joiner with String::join where simpler

2015-07-22 Thread kevints
Repository: aurora
Updated Branches:
  refs/heads/master fff3e38b4 - 38c2e769c


Replace Guava Joiner with String::join where simpler

Now that `String::join` is part of the standard library some uses of
Guava's `Joiner` can be removed.

Testing Done:
./gradlew -Pq build

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


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

Branch: refs/heads/master
Commit: 38c2e769c66cc26b590e067e7e56855cdefbc72f
Parents: fff3e38
Author: Kevin Sweeney kevi...@apache.org
Authored: Wed Jul 22 16:20:38 2015 -0700
Committer: Kevin Sweeney kevi...@apache.org
Committed: Wed Jul 22 16:20:38 2015 -0700

--
 .../apache/aurora/scheduler/base/JobKeys.java   |  3 +-
 .../aurora/scheduler/cron/CrontabEntry.java |  6 ++--
 .../aurora/scheduler/cron/quartz/Quartz.java|  3 +-
 .../aurora/scheduler/http/CorsFilter.java   | 30 +---
 .../thrift/aop/LoggingInterceptor.java  |  3 +-
 5 files changed, 21 insertions(+), 24 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/aurora/blob/38c2e769/src/main/java/org/apache/aurora/scheduler/base/JobKeys.java
--
diff --git a/src/main/java/org/apache/aurora/scheduler/base/JobKeys.java 
b/src/main/java/org/apache/aurora/scheduler/base/JobKeys.java
index a5ffa5e..38d2863 100644
--- a/src/main/java/org/apache/aurora/scheduler/base/JobKeys.java
+++ b/src/main/java/org/apache/aurora/scheduler/base/JobKeys.java
@@ -20,7 +20,6 @@ import javax.annotation.Nullable;
 
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
-import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
 import com.google.common.base.Splitter;
 import com.google.common.collect.ImmutableSet;
@@ -131,7 +130,7 @@ public final class JobKeys {
* @return Canonical /-delimited representation of the key.
*/
   public static String canonicalString(IJobKey jobKey) {
-return Joiner.on(/).join(jobKey.getRole(), jobKey.getEnvironment(), 
jobKey.getName());
+return String.join(/, jobKey.getRole(), jobKey.getEnvironment(), 
jobKey.getName());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/aurora/blob/38c2e769/src/main/java/org/apache/aurora/scheduler/cron/CrontabEntry.java
--
diff --git a/src/main/java/org/apache/aurora/scheduler/cron/CrontabEntry.java 
b/src/main/java/org/apache/aurora/scheduler/cron/CrontabEntry.java
index 904dd72..9d981da 100644
--- a/src/main/java/org/apache/aurora/scheduler/cron/CrontabEntry.java
+++ b/src/main/java/org/apache/aurora/scheduler/cron/CrontabEntry.java
@@ -21,7 +21,6 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
 import com.google.common.base.Splitter;
 import com.google.common.collect.BiMap;
@@ -218,7 +217,7 @@ public final class CrontabEntry {
 components.add(set.first() + - + set.last());
   }
 }
-return Joiner.on(,).join(components);
+return String.join(,, components);
   }
 
   /**
@@ -262,7 +261,8 @@ public final class CrontabEntry {
*/
   @Override
   public String toString() {
-return Joiner.on( ).join(
+return String.join(
+ ,
 getMinuteAsString(),
 getHourAsString(),
 getDayOfMonthAsString(),

http://git-wip-us.apache.org/repos/asf/aurora/blob/38c2e769/src/main/java/org/apache/aurora/scheduler/cron/quartz/Quartz.java
--
diff --git a/src/main/java/org/apache/aurora/scheduler/cron/quartz/Quartz.java 
b/src/main/java/org/apache/aurora/scheduler/cron/quartz/Quartz.java
index 45e062d..c379a32 100644
--- a/src/main/java/org/apache/aurora/scheduler/cron/quartz/Quartz.java
+++ b/src/main/java/org/apache/aurora/scheduler/cron/quartz/Quartz.java
@@ -69,7 +69,8 @@ final class Quartz {
   dayOfWeek = Joiner.on(,).join(daysOfWeek);
 }
 
-String rawCronExpresion = Joiner.on( ).join(
+String rawCronExpresion = String.join(
+ ,
 0,
 entry.getMinuteAsString(),
 entry.getHourAsString(),

http://git-wip-us.apache.org/repos/asf/aurora/blob/38c2e769/src/main/java/org/apache/aurora/scheduler/http/CorsFilter.java
--
diff --git a/src/main/java/org/apache/aurora/scheduler/http/CorsFilter.java 
b/src/main/java/org/apache/aurora/scheduler/http/CorsFilter.java
index e413ad9..58be4b7 100644
---