@Test
  public void testAddAndRemoveInstance() {
    try {
      Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext());

      Instance i1 = getRandomInstance(2);
      Instance i2 = getRandomInstance(2);
      Instance i3 = getRandomInstance(2);

      assertEquals(0, scheduler.getNumberOfAvailableInstances());
      assertEquals(0, scheduler.getNumberOfAvailableSlots());
      scheduler.newInstanceAvailable(i1);
      assertEquals(1, scheduler.getNumberOfAvailableInstances());
      assertEquals(2, scheduler.getNumberOfAvailableSlots());
      scheduler.newInstanceAvailable(i2);
      assertEquals(2, scheduler.getNumberOfAvailableInstances());
      assertEquals(4, scheduler.getNumberOfAvailableSlots());
      scheduler.newInstanceAvailable(i3);
      assertEquals(3, scheduler.getNumberOfAvailableInstances());
      assertEquals(6, scheduler.getNumberOfAvailableSlots());

      // cannot add available instance again
      try {
        scheduler.newInstanceAvailable(i2);
        fail("Scheduler accepted instance twice");
      } catch (IllegalArgumentException e) {
        // bueno!
      }

      // some instances die
      assertEquals(3, scheduler.getNumberOfAvailableInstances());
      assertEquals(6, scheduler.getNumberOfAvailableSlots());
      scheduler.instanceDied(i2);
      assertEquals(2, scheduler.getNumberOfAvailableInstances());
      assertEquals(4, scheduler.getNumberOfAvailableSlots());

      // try to add a dead instance
      try {
        scheduler.newInstanceAvailable(i2);
        fail("Scheduler accepted dead instance");
      } catch (IllegalArgumentException e) {
        // stimmt

      }

      scheduler.instanceDied(i1);
      assertEquals(1, scheduler.getNumberOfAvailableInstances());
      assertEquals(2, scheduler.getNumberOfAvailableSlots());
      scheduler.instanceDied(i3);
      assertEquals(0, scheduler.getNumberOfAvailableInstances());
      assertEquals(0, scheduler.getNumberOfAvailableSlots());

      assertFalse(i1.isAlive());
      assertFalse(i2.isAlive());
      assertFalse(i3.isAlive());
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void testScheduleQueueing() {
    final int NUM_INSTANCES = 50;
    final int NUM_SLOTS_PER_INSTANCE = 3;
    final int NUM_TASKS_TO_SCHEDULE = 2000;

    try {
      // note: since this test asynchronously releases slots, the executor needs release workers.
      // doing the release call synchronous can lead to a deadlock
      Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext());

      for (int i = 0; i < NUM_INSTANCES; i++) {
        scheduler.newInstanceAvailable(
            getRandomInstance((int) (Math.random() * NUM_SLOTS_PER_INSTANCE) + 1));
      }

      assertEquals(NUM_INSTANCES, scheduler.getNumberOfAvailableInstances());
      final int totalSlots = scheduler.getNumberOfAvailableSlots();

      // all slots we ever got.
      List<Future<SimpleSlot>> allAllocatedSlots = new ArrayList<>();

      // slots that need to be released
      final Set<SimpleSlot> toRelease = new HashSet<SimpleSlot>();

      // flag to track errors in the concurrent thread
      final AtomicBoolean errored = new AtomicBoolean(false);

      // thread to asynchronously release slots
      Runnable disposer =
          new Runnable() {

            @Override
            public void run() {
              try {
                int recycled = 0;
                while (recycled < NUM_TASKS_TO_SCHEDULE) {
                  synchronized (toRelease) {
                    while (toRelease.isEmpty()) {
                      toRelease.wait();
                    }

                    Iterator<SimpleSlot> iter = toRelease.iterator();
                    SimpleSlot next = iter.next();
                    iter.remove();

                    next.releaseSlot();
                    recycled++;
                  }
                }
              } catch (Throwable t) {
                errored.set(true);
              }
            }
          };

      Thread disposeThread = new Thread(disposer);
      disposeThread.start();

      for (int i = 0; i < NUM_TASKS_TO_SCHEDULE; i++) {
        Future<SimpleSlot> future = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), true);
        future.thenAcceptAsync(
            new AcceptFunction<SimpleSlot>() {
              @Override
              public void accept(SimpleSlot slot) {
                synchronized (toRelease) {
                  toRelease.add(slot);
                  toRelease.notifyAll();
                }
              }
            },
            TestingUtils.defaultExecutionContext());
        allAllocatedSlots.add(future);
      }

      disposeThread.join();

      assertFalse("The slot releasing thread caused an error.", errored.get());

      List<SimpleSlot> slotsAfter = new ArrayList<SimpleSlot>();
      for (Future<SimpleSlot> future : allAllocatedSlots) {
        slotsAfter.add(future.get());
      }

      assertEquals(
          "All instances should have available slots.",
          NUM_INSTANCES,
          scheduler.getNumberOfInstancesWithAvailableSlots());

      // the slots should all be different
      assertTrue(areAllDistinct(slotsAfter.toArray()));

      assertEquals(
          "All slots should be available.", totalSlots, scheduler.getNumberOfAvailableSlots());
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }