You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2014/10/30 02:37:19 UTC

git commit: Add FakeStatsProvider and check stat values in SchedulerLifecycleTest.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 0f63e8fc4 -> d85e616b3


Add FakeStatsProvider and check stat values in SchedulerLifecycleTest.

Bugs closed: AURORA-884

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


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

Branch: refs/heads/master
Commit: d85e616b3b930b4a3e55e362d13c98e51069145d
Parents: 0f63e8f
Author: Bill Farner <wf...@apache.org>
Authored: Wed Oct 29 18:36:52 2014 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Wed Oct 29 18:36:52 2014 -0700

----------------------------------------------------------------------
 .../aurora/scheduler/SchedulerLifecycle.java    | 52 ++++++++-----
 .../scheduler/SchedulerLifecycleTest.java       | 16 +++-
 .../scheduler/testing/FakeStatsProvider.java    | 78 ++++++++++++++++++++
 3 files changed, 127 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d85e616b/src/main/java/org/apache/aurora/scheduler/SchedulerLifecycle.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/SchedulerLifecycle.java b/src/main/java/org/apache/aurora/scheduler/SchedulerLifecycle.java
index 92ceaed..823ff75 100644
--- a/src/main/java/org/apache/aurora/scheduler/SchedulerLifecycle.java
+++ b/src/main/java/org/apache/aurora/scheduler/SchedulerLifecycle.java
@@ -27,6 +27,7 @@ import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
+import com.google.common.base.Supplier;
 import com.google.common.eventbus.Subscribe;
 import com.google.common.util.concurrent.Atomics;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -37,8 +38,7 @@ import com.twitter.common.base.Closures;
 import com.twitter.common.base.Command;
 import com.twitter.common.quantity.Amount;
 import com.twitter.common.quantity.Time;
-import com.twitter.common.stats.StatImpl;
-import com.twitter.common.stats.Stats;
+import com.twitter.common.stats.StatsProvider;
 import com.twitter.common.util.Clock;
 import com.twitter.common.util.StateMachine;
 import com.twitter.common.util.StateMachine.Transition;
@@ -93,7 +93,8 @@ public class SchedulerLifecycle implements EventSubscriber {
 
   private static final Logger LOG = Logger.getLogger(SchedulerLifecycle.class.getName());
 
-  private enum State {
+  @VisibleForTesting
+  enum State {
     IDLE,
     PREPARING_STORAGE,
     STORAGE_PREPARED,
@@ -132,7 +133,8 @@ public class SchedulerLifecycle implements EventSubscriber {
       ScheduledExecutorService executorService,
       Clock clock,
       EventSink eventSink,
-      ShutdownRegistry shutdownRegistry) {
+      ShutdownRegistry shutdownRegistry,
+      StatsProvider statsProvider) {
 
     this(
         driverFactory,
@@ -142,7 +144,8 @@ public class SchedulerLifecycle implements EventSubscriber {
         new DefaultDelayedActions(leadingOptions, executorService),
         clock,
         eventSink,
-        shutdownRegistry);
+        shutdownRegistry,
+        statsProvider);
   }
 
   private static final class DefaultDelayedActions implements DelayedActions {
@@ -192,6 +195,14 @@ public class SchedulerLifecycle implements EventSubscriber {
   }
 
   @VisibleForTesting
+  static final String REGISTERED_GAUGE = "framework_registered";
+
+  @VisibleForTesting
+  static String stateGaugeName(State state) {
+    return "scheduler_lifecycle_" + state;
+  }
+
+  @VisibleForTesting
   SchedulerLifecycle(
       final DriverFactory driverFactory,
       final NonVolatileStorage storage,
@@ -200,7 +211,8 @@ public class SchedulerLifecycle implements EventSubscriber {
       final DelayedActions delayedActions,
       final Clock clock,
       final EventSink eventSink,
-      final ShutdownRegistry shutdownRegistry) {
+      final ShutdownRegistry shutdownRegistry,
+      StatsProvider statsProvider) {
 
     requireNonNull(driverFactory);
     requireNonNull(storage);
@@ -211,19 +223,23 @@ public class SchedulerLifecycle implements EventSubscriber {
     requireNonNull(eventSink);
     requireNonNull(shutdownRegistry);
 
-    Stats.export(new StatImpl<Integer>("framework_registered") {
-      @Override
-      public Integer read() {
-        return registrationAcked.get() ? 1 : 0;
-      }
-    });
+    statsProvider.makeGauge(
+        REGISTERED_GAUGE,
+        new Supplier<Integer>() {
+          @Override
+          public Integer get() {
+            return registrationAcked.get() ? 1 : 0;
+          }
+        });
     for (final State state : State.values()) {
-      Stats.export(new StatImpl<Integer>("scheduler_lifecycle_" + state) {
-        @Override
-        public Integer read() {
-          return (state == stateMachine.getState()) ? 1 : 0;
-        }
-      });
+      statsProvider.makeGauge(
+          stateGaugeName(state),
+          new Supplier<Integer>() {
+            @Override
+            public Integer get() {
+              return (state == stateMachine.getState()) ? 1 : 0;
+            }
+          });
     }
 
     shutdownRegistry.addAction(new Command() {

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d85e616b/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java b/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java
index 90c8c06..d4d3a25 100644
--- a/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/SchedulerLifecycleTest.java
@@ -32,6 +32,7 @@ import org.apache.aurora.scheduler.events.PubsubEvent.SchedulerActive;
 import org.apache.aurora.scheduler.storage.Storage.MutateWork.NoResult.Quiet;
 import org.apache.aurora.scheduler.storage.Storage.StorageException;
 import org.apache.aurora.scheduler.storage.testing.StorageTestUtil;
+import org.apache.aurora.scheduler.testing.FakeStatsProvider;
 import org.apache.mesos.Protos.Status;
 import org.apache.mesos.SchedulerDriver;
 import org.easymock.Capture;
@@ -39,9 +40,12 @@ import org.easymock.EasyMock;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.aurora.scheduler.SchedulerLifecycle.State;
+import static org.apache.aurora.scheduler.SchedulerLifecycle.stateGaugeName;
 import static org.easymock.EasyMock.capture;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 public class SchedulerLifecycleTest extends EasyMockTest {
@@ -56,6 +60,7 @@ public class SchedulerLifecycleTest extends EasyMockTest {
   private SchedulerDriver schedulerDriver;
   private DelayedActions delayedActions;
   private EventSink eventSink;
+  private FakeStatsProvider statsProvider;
 
   private SchedulerLifecycle schedulerLifecycle;
 
@@ -69,6 +74,7 @@ public class SchedulerLifecycleTest extends EasyMockTest {
     schedulerDriver = createMock(SchedulerDriver.class);
     delayedActions = createMock(DelayedActions.class);
     eventSink = createMock(EventSink.class);
+    statsProvider = new FakeStatsProvider();
   }
 
   /**
@@ -98,7 +104,10 @@ public class SchedulerLifecycleTest extends EasyMockTest {
         delayedActions,
         clock,
         eventSink,
-        shutdownRegistry);
+        shutdownRegistry,
+        statsProvider);
+    assertEquals(0, statsProvider.getValue(SchedulerLifecycle.REGISTERED_GAUGE));
+    assertEquals(1, statsProvider.getValue(stateGaugeName(State.IDLE)));
     return shutdownCommand;
   }
 
@@ -149,8 +158,13 @@ public class SchedulerLifecycleTest extends EasyMockTest {
     replayAndCreateLifecycle();
 
     LeadershipListener leaderListener = schedulerLifecycle.prepare();
+    assertEquals(1, statsProvider.getValue(stateGaugeName(State.STORAGE_PREPARED)));
     leaderListener.onLeading(leaderControl);
+    assertEquals(1, statsProvider.getValue(stateGaugeName(State.LEADER_AWAITING_REGISTRATION)));
+    assertEquals(0, statsProvider.getValue(SchedulerLifecycle.REGISTERED_GAUGE));
     schedulerLifecycle.registered(new DriverRegistered());
+    assertEquals(1, statsProvider.getValue(stateGaugeName(State.ACTIVE)));
+    assertEquals(1, statsProvider.getValue(SchedulerLifecycle.REGISTERED_GAUGE));
     handleRegistered.getValue().run();
     triggerFailover.getValue().run();
   }

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/d85e616b/src/test/java/org/apache/aurora/scheduler/testing/FakeStatsProvider.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/testing/FakeStatsProvider.java b/src/test/java/org/apache/aurora/scheduler/testing/FakeStatsProvider.java
new file mode 100644
index 0000000..273129e
--- /dev/null
+++ b/src/test/java/org/apache/aurora/scheduler/testing/FakeStatsProvider.java
@@ -0,0 +1,78 @@
+/**
+ * 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.testing;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.Maps;
+import com.twitter.common.stats.Stat;
+import com.twitter.common.stats.StatsProvider;
+
+/**
+ * A fake stats provider for use in testing.
+ */
+public class FakeStatsProvider implements StatsProvider {
+  private final Map<String, Supplier<? extends Number>> stats = Maps.newHashMap();
+
+  /**
+   * Gets the current value of a stat.
+   *
+   * @param statName Name of the stat to fetch.
+   * @return Current stat value.
+   */
+  public Number getValue(String statName) {
+    return stats.get(statName).get();
+  }
+
+  @Override
+  public AtomicLong makeCounter(String name) {
+    final AtomicLong counter = new AtomicLong();
+    stats.put(name, new Supplier<Long>() {
+      @Override
+      public Long get() {
+        return counter.get();
+      }
+    });
+    return counter;
+  }
+
+  @Override
+  public <T extends Number> Stat<T> makeGauge(final String name, final Supplier<T> gauge) {
+    stats.put(name, gauge);
+
+    return new Stat<T>() {
+      @Override
+      public String getName() {
+        return name;
+      }
+
+      @Override
+      public T read() {
+        return gauge.get();
+      }
+    };
+  }
+
+  @Override
+  public StatsProvider untracked() {
+    return this;
+  }
+
+  @Override
+  public RequestTimer makeRequestTimer(String name) {
+    throw new UnsupportedOperationException();
+  }
+}