@Test
  public void testJobManagerProcessFailure() throws Exception {
    // Config
    final int numberOfJobManagers = 2;
    final int numberOfTaskManagers = 2;
    final int numberOfSlotsPerTaskManager = 2;

    assertEquals(PARALLELISM, numberOfTaskManagers * numberOfSlotsPerTaskManager);

    // Setup
    // Test actor system
    ActorSystem testActorSystem;

    // Job managers
    final JobManagerProcess[] jmProcess = new JobManagerProcess[numberOfJobManagers];

    // Task managers
    final ActorSystem[] tmActorSystem = new ActorSystem[numberOfTaskManagers];

    // Leader election service
    LeaderRetrievalService leaderRetrievalService = null;

    // Coordination between the processes goes through a directory
    File coordinateTempDir = null;

    try {
      final Deadline deadline = TestTimeOut.fromNow();

      // Coordination directory
      coordinateTempDir = createTempDirectory();

      // Job Managers
      Configuration config =
          ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(
              ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath());

      // Start first process
      jmProcess[0] = new JobManagerProcess(0, config);
      jmProcess[0].createAndStart();

      // Task manager configuration
      config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4);
      config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100);
      config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);

      // Start the task manager process
      for (int i = 0; i < numberOfTaskManagers; i++) {
        tmActorSystem[i] = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
        TaskManager.startTaskManagerComponentsAndActor(
            config,
            tmActorSystem[i],
            "localhost",
            Option.<String>empty(),
            Option.<LeaderRetrievalService>empty(),
            false,
            TaskManager.class);
      }

      // Test actor system
      testActorSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());

      jmProcess[0].getActorRef(testActorSystem, deadline.timeLeft());

      // Leader listener
      TestingListener leaderListener = new TestingListener();
      leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config);
      leaderRetrievalService.start(leaderListener);

      // Initial submission
      leaderListener.waitForNewLeader(deadline.timeLeft().toMillis());

      String leaderAddress = leaderListener.getAddress();
      UUID leaderId = leaderListener.getLeaderSessionID();

      // Get the leader ref
      ActorRef leaderRef =
          AkkaUtils.getActorRef(leaderAddress, testActorSystem, deadline.timeLeft());
      ActorGateway leaderGateway = new AkkaActorGateway(leaderRef, leaderId);

      // Wait for all task managers to connect to the leading job manager
      JobManagerActorTestUtils.waitForTaskManagers(
          numberOfTaskManagers, leaderGateway, deadline.timeLeft());

      final File coordinateDirClosure = coordinateTempDir;
      final Throwable[] errorRef = new Throwable[1];

      // we trigger program execution in a separate thread
      Thread programTrigger =
          new Thread("Program Trigger") {
            @Override
            public void run() {
              try {
                testJobManagerFailure(ZooKeeper.getConnectString(), coordinateDirClosure);
              } catch (Throwable t) {
                t.printStackTrace();
                errorRef[0] = t;
              }
            }
          };

      // start the test program
      programTrigger.start();

      // wait until all marker files are in place, indicating that all tasks have started
      AbstractTaskManagerProcessFailureRecoveryTest.waitForMarkerFiles(
          coordinateTempDir, READY_MARKER_FILE_PREFIX, PARALLELISM, deadline.timeLeft().toMillis());

      // Kill one of the job managers and trigger recovery
      jmProcess[0].destroy();

      jmProcess[1] = new JobManagerProcess(1, config);
      jmProcess[1].createAndStart();

      jmProcess[1].getActorRef(testActorSystem, deadline.timeLeft());

      // we create the marker file which signals the program functions tasks that they can complete
      AbstractTaskManagerProcessFailureRecoveryTest.touchFile(
          new File(coordinateTempDir, PROCEED_MARKER_FILE));

      programTrigger.join(deadline.timeLeft().toMillis());

      // We wait for the finish marker file. We don't wait for the program trigger, because
      // we submit in detached mode.
      AbstractTaskManagerProcessFailureRecoveryTest.waitForMarkerFiles(
          coordinateTempDir, FINISH_MARKER_FILE_PREFIX, 1, deadline.timeLeft().toMillis());

      // check that the program really finished
      assertFalse("The program did not finish in time", programTrigger.isAlive());

      // check whether the program encountered an error
      if (errorRef[0] != null) {
        Throwable error = errorRef[0];
        error.printStackTrace();
        fail(
            "The program encountered a "
                + error.getClass().getSimpleName()
                + " : "
                + error.getMessage());
      }
    } catch (Throwable t) {
      // Print early (in some situations the process logs get too big
      // for Travis and the root problem is not shown)
      t.printStackTrace();

      for (JobManagerProcess p : jmProcess) {
        if (p != null) {
          p.printProcessLog();
        }
      }

      throw t;
    } finally {
      for (int i = 0; i < numberOfTaskManagers; i++) {
        if (tmActorSystem[i] != null) {
          tmActorSystem[i].shutdown();
        }
      }

      if (leaderRetrievalService != null) {
        leaderRetrievalService.stop();
      }

      for (JobManagerProcess jmProces : jmProcess) {
        if (jmProces != null) {
          jmProces.destroy();
        }
      }

      // Delete coordination directory
      if (coordinateTempDir != null) {
        try {
          FileUtils.deleteDirectory(coordinateTempDir);
        } catch (Throwable ignored) {
        }
      }
    }
  }
Пример #2
0
  /**
   * 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.");
        }
      }
    }
  }
Пример #3
0
  /**
   * 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.");
        }
      }
    }
  }
Пример #4
0
  /**
   * 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.");
        }
      }
    }
  }