/**
   * Tests that a job is properly canceled in the event of a leader change. However, this time only
   * the TMs are notified about the changing leader. This should be enough to cancel the currently
   * running job, though.
   */
  @Test
  public void testStateCleanupAfterListenerNotification() throws Exception {
    UUID leaderSessionID = UUID.randomUUID();
    UUID newLeaderSessionID = UUID.randomUUID();

    cluster.grantLeadership(0, leaderSessionID);
    cluster.notifyRetrievalListeners(0, leaderSessionID);

    cluster.waitForTaskManagersToBeRegistered();

    // submit blocking job
    cluster.submitJobDetached(job);

    ActorGateway jm = cluster.getLeaderGateway(timeout);

    Future<Object> wait =
        jm.ask(new WaitForAllVerticesToBeRunningOrFinished(job.getJobID()), timeout);

    Await.ready(wait, timeout);

    Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);

    // notify listeners (TMs) about the leader change
    cluster.notifyRetrievalListeners(1, newLeaderSessionID);

    Await.ready(jobRemoval, timeout);
  }
  private static String[] runTests(
      int numHosts, int slotsPerHost, int parallelism, TestLocatableInputSplit[] splits)
      throws Exception {
    AbstractJobVertex vertex = new AbstractJobVertex("test vertex");
    vertex.setParallelism(parallelism);
    vertex.setInvokableClass(DummyInvokable.class);
    vertex.setInputSplitSource(new TestInputSplitSource(splits));

    JobGraph jobGraph = new JobGraph("test job", vertex);

    ExecutionGraph eg =
        new ExecutionGraph(
            jobGraph.getJobID(), jobGraph.getName(), jobGraph.getJobConfiguration(), TIMEOUT);
    eg.setQueuedSchedulingAllowed(false);

    eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources());

    Scheduler scheduler = getScheduler(numHosts, slotsPerHost);
    eg.scheduleForExecution(scheduler);

    ExecutionVertex[] tasks = eg.getVerticesTopologically().iterator().next().getTaskVertices();
    assertEquals(parallelism, tasks.length);

    String[] hostsForTasks = new String[parallelism];
    for (int i = 0; i < parallelism; i++) {
      hostsForTasks[i] = tasks[i].getCurrentAssignedResourceLocation().getHostname();
    }

    return hostsForTasks;
  }
  /**
   * Tests that a job is properly canceled in the case of a leader change. However, this time only
   * the JMs are notified about the leader change and the TMs still believe the old leader to have
   * leadership.
   */
  @Test
  public void testStateCleanupAfterNewLeaderElection() throws Exception {
    UUID leaderSessionID = UUID.randomUUID();
    UUID newLeaderSessionID = UUID.randomUUID();

    cluster.grantLeadership(0, leaderSessionID);
    cluster.notifyRetrievalListeners(0, leaderSessionID);

    cluster.waitForTaskManagersToBeRegistered();

    // submit blocking job so that we can test job clean up
    cluster.submitJobDetached(job);

    ActorGateway jm = cluster.getLeaderGateway(timeout);

    Future<Object> wait =
        jm.ask(new WaitForAllVerticesToBeRunningOrFinished(job.getJobID()), timeout);

    Await.ready(wait, timeout);

    Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);

    // only notify the JMs about the new leader JM(1)
    cluster.grantLeadership(1, newLeaderSessionID);

    // job should be removed anyway
    Await.ready(jobRemoval, timeout);
  }
  private JobGraph createTestJobGraph(
      String jobName, int senderParallelism, int receiverParallelism) {

    // The sender and receiver invokable logic ensure that each subtask gets the expected data
    final JobVertex sender = new JobVertex("Sender");
    sender.setInvokableClass(RoundRobinSubtaskIndexSender.class);
    sender
        .getConfiguration()
        .setInteger(RoundRobinSubtaskIndexSender.CONFIG_KEY, receiverParallelism);
    sender.setParallelism(senderParallelism);

    final JobVertex receiver = new JobVertex("Receiver");
    receiver.setInvokableClass(SubtaskIndexReceiver.class);
    receiver.getConfiguration().setInteger(SubtaskIndexReceiver.CONFIG_KEY, senderParallelism);
    receiver.setParallelism(receiverParallelism);

    receiver.connectNewDataSetAsInput(
        sender, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING);

    final JobGraph jobGraph = new JobGraph(jobName, sender, receiver);

    // We need to allow queued scheduling, because there are not enough slots available
    // to run all tasks at once. We queue tasks and then let them finish/consume the blocking
    // result one after the other.
    jobGraph.setAllowQueuedScheduling(true);

    return jobGraph;
  }
Exemple #5
0
  /**
   * Parameter {@code uploadedJarLocation} is actually used to point to the local jar, because Flink
   * does not support uploading a jar file before hand. Jar files are always uploaded directly when
   * a program is submitted.
   */
  public void submitTopologyWithOpts(
      final String name, final String uploadedJarLocation, final FlinkTopology topology)
      throws AlreadyAliveException, InvalidTopologyException {

    if (this.getTopologyJobId(name) != null) {
      throw new AlreadyAliveException();
    }

    final URI uploadedJarUri;
    final URL uploadedJarUrl;
    try {
      uploadedJarUri = new File(uploadedJarLocation).getAbsoluteFile().toURI();
      uploadedJarUrl = uploadedJarUri.toURL();
      JobWithJars.checkJarFile(uploadedJarUrl);
    } catch (final IOException e) {
      throw new RuntimeException("Problem with jar file " + uploadedJarLocation, e);
    }

    try {
      FlinkClient.addStormConfigToTopology(topology, conf);
    } catch (ClassNotFoundException e) {
      LOG.error("Could not register class for Kryo serialization.", e);
      throw new InvalidTopologyException("Could not register class for Kryo serialization.");
    }

    final StreamGraph streamGraph = topology.getExecutionEnvironment().getStreamGraph();
    streamGraph.setJobName(name);

    final JobGraph jobGraph = streamGraph.getJobGraph();
    jobGraph.addJar(new Path(uploadedJarUri));

    final Configuration configuration = jobGraph.getJobConfiguration();
    configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerHost);
    configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);

    final Client client;
    try {
      client = new Client(configuration);
    } catch (IOException e) {
      throw new RuntimeException("Could not establish a connection to the job manager", e);
    }

    try {
      ClassLoader classLoader =
          JobWithJars.buildUserCodeClassLoader(
              Lists.newArrayList(uploadedJarUrl),
              Collections.<URL>emptyList(),
              this.getClass().getClassLoader());
      client.runDetached(jobGraph, classLoader);
    } catch (final ProgramInvocationException e) {
      throw new RuntimeException("Cannot execute job due to ProgramInvocationException", e);
    }
  }
  /**
   * Tests that a job is properly canceled in the case of a leader change. In such an event all
   * TaskManagers have to disconnect from the previous leader and connect to the newly elected
   * leader.
   */
  @Test
  public void testStateCleanupAfterNewLeaderElectionAndListenerNotification() throws Exception {
    UUID leaderSessionID1 = UUID.randomUUID();
    UUID leaderSessionID2 = UUID.randomUUID();

    // first make JM(0) the leader
    cluster.grantLeadership(0, leaderSessionID1);
    // notify all listeners
    cluster.notifyRetrievalListeners(0, leaderSessionID1);

    cluster.waitForTaskManagersToBeRegistered();

    // submit blocking job so that it is not finished when we cancel it
    cluster.submitJobDetached(job);

    ActorGateway jm = cluster.getLeaderGateway(timeout);

    Future<Object> wait =
        jm.ask(new WaitForAllVerticesToBeRunningOrFinished(job.getJobID()), timeout);

    Await.ready(wait, timeout);

    Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);

    // make the JM(1) the new leader
    cluster.grantLeadership(1, leaderSessionID2);
    // notify all listeners about the event
    cluster.notifyRetrievalListeners(1, leaderSessionID2);

    Await.ready(jobRemoval, timeout);

    cluster.waitForTaskManagersToBeRegistered();

    ActorGateway jm2 = cluster.getLeaderGateway(timeout);

    Future<Object> futureNumberSlots =
        jm2.ask(JobManagerMessages.getRequestTotalNumberOfSlots(), timeout);

    // check that all TMs have registered at the new leader
    int numberSlots = (Integer) Await.result(futureNumberSlots, timeout);

    assertEquals(parallelism, numberSlots);

    // try to resubmit now the non-blocking job, it should complete successfully
    Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
    cluster.submitJobAndWait(job, false, timeout);
  }
  /**
   * Tests that the same JobManager can be reelected as the leader. Even though, the same JM is
   * elected as the next leader, all currently running jobs should be canceled properly and all TMs
   * should disconnect from the leader and then reconnect to it.
   */
  @Test
  public void testReelectionOfSameJobManager() throws Exception {
    UUID leaderSessionID = UUID.randomUUID();
    UUID newLeaderSessionID = UUID.randomUUID();

    FiniteDuration shortTimeout = new FiniteDuration(20, TimeUnit.SECONDS);

    cluster.grantLeadership(0, leaderSessionID);
    cluster.notifyRetrievalListeners(0, leaderSessionID);

    cluster.waitForTaskManagersToBeRegistered();

    // submit blocking job
    cluster.submitJobDetached(job);

    ActorGateway jm = cluster.getLeaderGateway(timeout);

    Future<Object> wait =
        jm.ask(new WaitForAllVerticesToBeRunningOrFinished(job.getJobID()), timeout);

    Await.ready(wait, timeout);

    Future<Object> jobRemoval = jm.ask(new NotifyWhenJobRemoved(job.getJobID()), timeout);

    // make JM(0) again the leader --> this implies first a leadership revokal
    cluster.grantLeadership(0, newLeaderSessionID);

    Await.ready(jobRemoval, timeout);

    // The TMs should not be able to reconnect since they don't know the current leader
    // session ID
    try {
      cluster.waitForTaskManagersToBeRegistered(shortTimeout);
      fail("TaskManager should not be able to register at JobManager.");
    } catch (TimeoutException e) {
      // expected exception since the TMs have still the old leader session ID
    }

    // notify the TMs about the new (old) leader
    cluster.notifyRetrievalListeners(0, newLeaderSessionID);

    cluster.waitForTaskManagersToBeRegistered();

    // try to resubmit now the non-blocking job, it should complete successfully
    Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false);
    cluster.submitJobAndWait(job, false, timeout);
  }
  @Test
  public void testFailureWhenJarBlobsMissing() {
    try {
      // create a simple job graph
      JobVertex jobVertex = new JobVertex("Test Vertex");
      jobVertex.setInvokableClass(Tasks.NoOpInvokable.class);
      JobGraph jg = new JobGraph("test job", jobVertex);

      // request the blob port from the job manager
      Future<Object> future =
          jmGateway.ask(JobManagerMessages.getRequestBlobManagerPort(), timeout);
      int blobPort = (Integer) Await.result(future, timeout);

      // upload two dummy bytes and add their keys to the job graph as dependencies
      BlobKey key1, key2;
      BlobClient bc = new BlobClient(new InetSocketAddress("localhost", blobPort));
      try {
        key1 = bc.put(new byte[10]);
        key2 = bc.put(new byte[10]);

        // delete one of the blobs to make sure that the startup failed
        bc.delete(key2);
      } finally {
        bc.close();
      }

      jg.addBlob(key1);
      jg.addBlob(key2);

      // submit the job
      Future<Object> submitFuture =
          jmGateway.ask(
              new JobManagerMessages.SubmitJob(jg, ListeningBehaviour.EXECUTION_RESULT), timeout);
      try {
        Await.result(submitFuture, timeout);
      } catch (JobExecutionException e) {
        // that is what we expect
        assertTrue(e.getCause() instanceof IOException);
      } catch (Exception e) {
        fail("Wrong exception type");
      }
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  public JobManagerJobMetricGroup addJob(JobGraph job) {
    JobID jobId = job.getJobID();
    String jobName = job.getName();
    // get or create a jobs metric group
    JobManagerJobMetricGroup currentJobGroup;
    synchronized (this) {
      if (!isClosed()) {
        currentJobGroup = jobs.get(jobId);

        if (currentJobGroup == null || currentJobGroup.isClosed()) {
          currentJobGroup = new JobManagerJobMetricGroup(registry, this, jobId, jobName);
          jobs.put(jobId, currentJobGroup);
        }
        return currentJobGroup;
      } else {
        return null;
      }
    }
  }
  @Test
  public void testMultipleInstancesPerHost() {

    TestLocatableInputSplit[] splits =
        new TestLocatableInputSplit[] {
          new TestLocatableInputSplit(1, "host1"),
          new TestLocatableInputSplit(2, "host1"),
          new TestLocatableInputSplit(3, "host2"),
          new TestLocatableInputSplit(4, "host2"),
          new TestLocatableInputSplit(5, "host3"),
          new TestLocatableInputSplit(6, "host3")
        };

    try {
      AbstractJobVertex vertex = new AbstractJobVertex("test vertex");
      vertex.setParallelism(6);
      vertex.setInvokableClass(DummyInvokable.class);
      vertex.setInputSplitSource(new TestInputSplitSource(splits));

      JobGraph jobGraph = new JobGraph("test job", vertex);

      ExecutionGraph eg =
          new ExecutionGraph(
              jobGraph.getJobID(), jobGraph.getName(), jobGraph.getJobConfiguration(), TIMEOUT);

      eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources());
      eg.setQueuedSchedulingAllowed(false);

      // create a scheduler with 6 instances where always two are on the same host
      Scheduler scheduler = new Scheduler();
      Instance i1 = getInstance(new byte[] {10, 0, 1, 1}, 12345, "host1", 1);
      Instance i2 = getInstance(new byte[] {10, 0, 1, 1}, 12346, "host1", 1);
      Instance i3 = getInstance(new byte[] {10, 0, 1, 2}, 12345, "host2", 1);
      Instance i4 = getInstance(new byte[] {10, 0, 1, 2}, 12346, "host2", 1);
      Instance i5 = getInstance(new byte[] {10, 0, 1, 3}, 12345, "host3", 1);
      Instance i6 = getInstance(new byte[] {10, 0, 1, 3}, 12346, "host4", 1);
      scheduler.newInstanceAvailable(i1);
      scheduler.newInstanceAvailable(i2);
      scheduler.newInstanceAvailable(i3);
      scheduler.newInstanceAvailable(i4);
      scheduler.newInstanceAvailable(i5);
      scheduler.newInstanceAvailable(i6);

      eg.scheduleForExecution(scheduler);

      ExecutionVertex[] tasks = eg.getVerticesTopologically().iterator().next().getTaskVertices();
      assertEquals(6, tasks.length);

      Instance taskInstance1 = tasks[0].getCurrentAssignedResource().getInstance();
      Instance taskInstance2 = tasks[1].getCurrentAssignedResource().getInstance();
      Instance taskInstance3 = tasks[2].getCurrentAssignedResource().getInstance();
      Instance taskInstance4 = tasks[3].getCurrentAssignedResource().getInstance();
      Instance taskInstance5 = tasks[4].getCurrentAssignedResource().getInstance();
      Instance taskInstance6 = tasks[5].getCurrentAssignedResource().getInstance();

      assertTrue(taskInstance1 == i1 || taskInstance1 == i2);
      assertTrue(taskInstance2 == i1 || taskInstance2 == i2);
      assertTrue(taskInstance3 == i3 || taskInstance3 == i4);
      assertTrue(taskInstance4 == i3 || taskInstance4 == i4);
      assertTrue(taskInstance5 == i5 || taskInstance5 == i6);
      assertTrue(taskInstance6 == i5 || taskInstance6 == i6);
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  protected void internalNonBlockingStart() throws IOException {

    Plan plan = env.createProgramPlan();

    Optimizer optimizer =
        new Optimizer(new DataStatistics(), new org.apache.flink.configuration.Configuration());
    OptimizedPlan optimizedPlan = optimizer.compile(plan);

    final JobGraph jobGraph = new JobGraphGenerator().compileJobGraph(optimizedPlan);
    for (String jarPath : classPath) {
      jobGraph.addJar(new Path(jarPath));
    }

    jobID = jobGraph.getJobID();
    accumulatorCache.setJobID(jobID);

    if (isLocalExecution()) {

      flowStep.logInfo("Executing in local mode.");

      startLocalCluster();

      org.apache.flink.configuration.Configuration config =
          new org.apache.flink.configuration.Configuration();
      config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, localCluster.hostname());

      client = new Client(config);
      client.setPrintStatusDuringExecution(env.getConfig().isSysoutLoggingEnabled());

    } else {

      flowStep.logInfo("Executing in cluster mode.");

      try {
        String path =
            this.getClass().getProtectionDomain().getCodeSource().getLocation().toURI().getPath();
        jobGraph.addJar(new Path(path));
        classPath.add(path);
      } catch (URISyntaxException e) {
        throw new IOException("Could not add the submission JAR as a dependency.");
      }

      client = ((ContextEnvironment) env).getClient();
    }

    List<URL> fileList = new ArrayList<URL>(classPath.size());
    for (String path : classPath) {
      URL url;
      try {
        url = new URL(path);
      } catch (MalformedURLException e) {
        url = new URL("file://" + path);
      }
      fileList.add(url);
    }

    final ClassLoader loader =
        JobWithJars.buildUserCodeClassLoader(
            fileList, Collections.<URL>emptyList(), getClass().getClassLoader());

    accumulatorCache.setClient(client);

    final Callable<JobSubmissionResult> callable =
        new Callable<JobSubmissionResult>() {
          @Override
          public JobSubmissionResult call() throws Exception {
            return client.runBlocking(jobGraph, loader);
          }
        };

    jobSubmission = executorService.submit(callable);

    flowStep.logInfo("submitted Flink job: " + jobID);
  }
  /**
   * Tests rescaling of partitioned operator state. More specific, we test the mechanism with {@link
   * ListCheckpointed} as it subsumes {@link
   * org.apache.flink.streaming.api.checkpoint.CheckpointedFunction}.
   */
  public void testSavepointRescalingPartitionedOperatorState(boolean scaleOut) throws Exception {
    final int parallelism = scaleOut ? numSlots : numSlots / 2;
    final int parallelism2 = scaleOut ? numSlots / 2 : numSlots;
    final int maxParallelism = 13;

    FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES);
    Deadline deadline = timeout.fromNow();

    JobID jobID = null;
    ActorGateway jobManager = null;

    int counterSize = Math.max(parallelism, parallelism2);

    PartitionedStateSource.CHECK_CORRECT_SNAPSHOT = new int[counterSize];
    PartitionedStateSource.CHECK_CORRECT_RESTORE = new int[counterSize];

    try {
      jobManager = cluster.getLeaderGateway(deadline.timeLeft());

      JobGraph jobGraph = createJobGraphWithOperatorState(parallelism, maxParallelism, true);

      jobID = jobGraph.getJobID();

      cluster.submitJobDetached(jobGraph);

      Object savepointResponse = null;

      // wait until the operator is started
      StateSourceBase.workStartedLatch.await();

      while (deadline.hasTimeLeft()) {

        Future<Object> savepointPathFuture =
            jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID), deadline.timeLeft());
        FiniteDuration waitingTime = new FiniteDuration(10, TimeUnit.SECONDS);
        savepointResponse = Await.result(savepointPathFuture, waitingTime);

        if (savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess) {
          break;
        }
      }

      assertTrue(savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess);

      final String savepointPath =
          ((JobManagerMessages.TriggerSavepointSuccess) savepointResponse).savepointPath();

      Future<Object> jobRemovedFuture =
          jobManager.ask(
              new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), deadline.timeLeft());

      Future<Object> cancellationResponseFuture =
          jobManager.ask(new JobManagerMessages.CancelJob(jobID), deadline.timeLeft());

      Object cancellationResponse = Await.result(cancellationResponseFuture, deadline.timeLeft());

      assertTrue(cancellationResponse instanceof JobManagerMessages.CancellationSuccess);

      Await.ready(jobRemovedFuture, deadline.timeLeft());

      // job successfully removed
      jobID = null;

      JobGraph scaledJobGraph = createJobGraphWithOperatorState(parallelism2, maxParallelism, true);

      scaledJobGraph.setSavepointPath(savepointPath);

      jobID = scaledJobGraph.getJobID();

      cluster.submitJobAndWait(scaledJobGraph, false);

      int sumExp = 0;
      int sumAct = 0;

      for (int c : PartitionedStateSource.CHECK_CORRECT_SNAPSHOT) {
        sumExp += c;
      }

      for (int c : PartitionedStateSource.CHECK_CORRECT_RESTORE) {
        sumAct += c;
      }

      assertEquals(sumExp, sumAct);
      jobID = null;

    } finally {
      // clear any left overs from a possibly failed job
      if (jobID != null && jobManager != null) {
        Future<Object> jobRemovedFuture =
            jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), timeout);

        try {
          Await.ready(jobRemovedFuture, timeout);
        } catch (TimeoutException | InterruptedException ie) {
          fail("Failed while cleaning up the cluster.");
        }
      }
    }
  }
  /**
   * Tests that a job with non partitioned state can be restarted from a savepoint with a different
   * parallelism if the operator with non-partitioned state are not rescaled.
   *
   * @throws Exception
   */
  @Test
  public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Exception {
    int numberKeys = 42;
    int numberElements = 1000;
    int numberElements2 = 500;
    int parallelism = numSlots / 2;
    int parallelism2 = numSlots;
    int maxParallelism = 13;

    FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES);
    Deadline deadline = timeout.fromNow();

    ActorGateway jobManager = null;
    JobID jobID = null;

    try {
      jobManager = cluster.getLeaderGateway(deadline.timeLeft());

      JobGraph jobGraph =
          createJobGraphWithKeyedAndNonPartitionedOperatorState(
              parallelism, maxParallelism, parallelism, numberKeys, numberElements, false, 100);

      jobID = jobGraph.getJobID();

      cluster.submitJobDetached(jobGraph);

      // wait til the sources have emitted numberElements for each key and completed a checkpoint
      SubtaskIndexFlatMapper.workCompletedLatch.await(
          deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);

      // verify the current state

      Set<Tuple2<Integer, Integer>> actualResult = CollectionSink.getElementsSet();

      Set<Tuple2<Integer, Integer>> expectedResult = new HashSet<>();

      for (int key = 0; key < numberKeys; key++) {
        int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism);

        expectedResult.add(
            Tuple2.of(
                KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(
                    maxParallelism, parallelism, keyGroupIndex),
                numberElements * key));
      }

      assertEquals(expectedResult, actualResult);

      // clear the CollectionSink set for the restarted job
      CollectionSink.clearElementsSet();

      Future<Object> savepointPathFuture =
          jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID), deadline.timeLeft());

      final String savepointPath =
          ((JobManagerMessages.TriggerSavepointSuccess)
                  Await.result(savepointPathFuture, deadline.timeLeft()))
              .savepointPath();

      Future<Object> jobRemovedFuture =
          jobManager.ask(
              new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), deadline.timeLeft());

      Future<Object> cancellationResponseFuture =
          jobManager.ask(new JobManagerMessages.CancelJob(jobID), deadline.timeLeft());

      Object cancellationResponse = Await.result(cancellationResponseFuture, deadline.timeLeft());

      assertTrue(cancellationResponse instanceof JobManagerMessages.CancellationSuccess);

      Await.ready(jobRemovedFuture, deadline.timeLeft());

      jobID = null;

      JobGraph scaledJobGraph =
          createJobGraphWithKeyedAndNonPartitionedOperatorState(
              parallelism2,
              maxParallelism,
              parallelism,
              numberKeys,
              numberElements + numberElements2,
              true,
              100);

      scaledJobGraph.setSavepointPath(savepointPath);

      jobID = scaledJobGraph.getJobID();

      cluster.submitJobAndWait(scaledJobGraph, false);

      jobID = null;

      Set<Tuple2<Integer, Integer>> actualResult2 = CollectionSink.getElementsSet();

      Set<Tuple2<Integer, Integer>> expectedResult2 = new HashSet<>();

      for (int key = 0; key < numberKeys; key++) {
        int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(key, maxParallelism);
        expectedResult2.add(
            Tuple2.of(
                KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(
                    maxParallelism, parallelism2, keyGroupIndex),
                key * (numberElements + numberElements2)));
      }

      assertEquals(expectedResult2, actualResult2);

    } finally {
      // clear the CollectionSink set for the restarted job
      CollectionSink.clearElementsSet();

      // clear any left overs from a possibly failed job
      if (jobID != null && jobManager != null) {
        Future<Object> jobRemovedFuture =
            jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), timeout);

        try {
          Await.ready(jobRemovedFuture, timeout);
        } catch (TimeoutException | InterruptedException ie) {
          fail("Failed while cleaning up the cluster.");
        }
      }
    }
  }
  /**
   * Tests that a job cannot be restarted from a savepoint with a different parallelism if the
   * rescaled operator has non-partitioned state.
   *
   * @throws Exception
   */
  @Test
  public void testSavepointRescalingNonPartitionedStateCausesException() throws Exception {
    final int parallelism = numSlots / 2;
    final int parallelism2 = numSlots;
    final int maxParallelism = 13;

    FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES);
    Deadline deadline = timeout.fromNow();

    JobID jobID = null;
    ActorGateway jobManager = null;

    try {
      jobManager = cluster.getLeaderGateway(deadline.timeLeft());

      JobGraph jobGraph = createJobGraphWithOperatorState(parallelism, maxParallelism, false);

      jobID = jobGraph.getJobID();

      cluster.submitJobDetached(jobGraph);

      Object savepointResponse = null;

      // wait until the operator is started
      StateSourceBase.workStartedLatch.await();

      while (deadline.hasTimeLeft()) {

        Future<Object> savepointPathFuture =
            jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID), deadline.timeLeft());
        FiniteDuration waitingTime = new FiniteDuration(10, TimeUnit.SECONDS);
        savepointResponse = Await.result(savepointPathFuture, waitingTime);

        if (savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess) {
          break;
        }
      }

      assertTrue(savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess);

      final String savepointPath =
          ((JobManagerMessages.TriggerSavepointSuccess) savepointResponse).savepointPath();

      Future<Object> jobRemovedFuture =
          jobManager.ask(
              new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), deadline.timeLeft());

      Future<Object> cancellationResponseFuture =
          jobManager.ask(new JobManagerMessages.CancelJob(jobID), deadline.timeLeft());

      Object cancellationResponse = Await.result(cancellationResponseFuture, deadline.timeLeft());

      assertTrue(cancellationResponse instanceof JobManagerMessages.CancellationSuccess);

      Await.ready(jobRemovedFuture, deadline.timeLeft());

      // job successfully removed
      jobID = null;

      JobGraph scaledJobGraph =
          createJobGraphWithOperatorState(parallelism2, maxParallelism, false);

      scaledJobGraph.setSavepointPath(savepointPath);

      jobID = scaledJobGraph.getJobID();

      cluster.submitJobAndWait(scaledJobGraph, false);

      jobID = null;

    } catch (JobExecutionException exception) {
      if (exception.getCause() instanceof SuppressRestartsException) {
        SuppressRestartsException suppressRestartsException =
            (SuppressRestartsException) exception.getCause();

        if (suppressRestartsException.getCause() instanceof IllegalStateException) {
          // we expect a IllegalStateException wrapped in a SuppressRestartsException wrapped
          // in a JobExecutionException, because the job containing non-partitioned state
          // is being rescaled
        } else {
          throw exception;
        }
      } else {
        throw exception;
      }
    } finally {
      // clear any left overs from a possibly failed job
      if (jobID != null && jobManager != null) {
        Future<Object> jobRemovedFuture =
            jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), timeout);

        try {
          Await.ready(jobRemovedFuture, timeout);
        } catch (TimeoutException | InterruptedException ie) {
          fail("Failed while cleaning up the cluster.");
        }
      }
    }
  }