Exemplo n.º 1
0
  @Test(timeout = 10000)
  public void testEventThrottling() throws Exception {
    TezTaskAttemptID mockTaskAttemptId = mock(TezTaskAttemptID.class);
    LogicalIOProcessorRuntimeTask mockTask = mock(LogicalIOProcessorRuntimeTask.class);
    when(mockTask.getMaxEventsToHandle()).thenReturn(10000, 1);
    when(mockTask.getVertexName()).thenReturn("vertexName");
    when(mockTask.getTaskAttemptID()).thenReturn(mockTaskAttemptId);

    TezTaskUmbilicalProtocol mockUmbilical = mock(TezTaskUmbilicalProtocol.class);
    TezHeartbeatResponse resp1 = new TezHeartbeatResponse(createEvents(5));
    resp1.setLastRequestId(1);
    TezHeartbeatResponse resp2 = new TezHeartbeatResponse(createEvents(1));
    resp2.setLastRequestId(2);
    resp2.setShouldDie();
    when(mockUmbilical.heartbeat(isA(TezHeartbeatRequest.class))).thenReturn(resp1, resp2);

    // Setup the sleep time to be way higher than the test timeout
    TaskReporter.HeartbeatCallable heartbeatCallable =
        new TaskReporter.HeartbeatCallable(
            mockTask, mockUmbilical, 100000, 100000, 5, new AtomicLong(0), "containerIdStr");

    ExecutorService executor = Executors.newSingleThreadExecutor();
    try {
      Future<Boolean> result = executor.submit(heartbeatCallable);
      Assert.assertFalse(result.get());
    } finally {
      executor.shutdownNow();
    }

    ArgumentCaptor<TezHeartbeatRequest> captor = ArgumentCaptor.forClass(TezHeartbeatRequest.class);
    verify(mockUmbilical, times(2)).heartbeat(captor.capture());
    TezHeartbeatRequest req = captor.getValue();
    Assert.assertEquals(2, req.getRequestId());
    Assert.assertEquals(1, req.getMaxEvents());
  }
  @Test
  public void testReduceProcessor() throws Exception {
    final String dagName = "mrdag0";
    String mapVertexName = MultiStageMRConfigUtil.getInitialMapVertexName();
    String reduceVertexName = MultiStageMRConfigUtil.getFinalReduceVertexName();
    JobConf jobConf = new JobConf(defaultConf);
    setUpJobConf(jobConf);

    MRHelpers.translateVertexConfToTez(jobConf);
    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0);

    jobConf.set(
        MRFrameworkConfigs.TASK_LOCAL_RESOURCE_DIR,
        new Path(workDir, "localized-resources").toUri().toString());
    jobConf.setBoolean(MRJobConfig.MR_TEZ_SPLITS_VIA_EVENTS, false);

    Path mapInput = new Path(workDir, "map0");
    MapUtils.generateInputSplit(localFs, workDir, jobConf, mapInput);

    InputSpec mapInputSpec =
        new InputSpec(
            "NullSrcVertex",
            new InputDescriptor(MRInputLegacy.class.getName())
                .setUserPayload(MRHelpers.createMRInputPayload(jobConf, null)),
            1);
    OutputSpec mapOutputSpec =
        new OutputSpec(
            "NullDestVertex",
            new OutputDescriptor(LocalOnFileSorterOutput.class.getName())
                .setUserPayload(TezUtils.createUserPayloadFromConf(jobConf)),
            1);
    // Run a map
    LogicalIOProcessorRuntimeTask mapTask =
        MapUtils.createLogicalTask(
            localFs,
            workDir,
            jobConf,
            0,
            mapInput,
            new TestUmbilical(),
            dagName,
            mapVertexName,
            Collections.singletonList(mapInputSpec),
            Collections.singletonList(mapOutputSpec));

    mapTask.initialize();
    mapTask.run();
    mapTask.close();

    LOG.info("Starting reduce...");

    Token<JobTokenIdentifier> shuffleToken = new Token<JobTokenIdentifier>();

    jobConf.setOutputFormat(SequenceFileOutputFormat.class);
    jobConf.set(
        MRFrameworkConfigs.TASK_LOCAL_RESOURCE_DIR,
        new Path(workDir, "localized-resources").toUri().toString());
    FileOutputFormat.setOutputPath(jobConf, new Path(workDir, "output"));
    ProcessorDescriptor reduceProcessorDesc =
        new ProcessorDescriptor(ReduceProcessor.class.getName())
            .setUserPayload(TezUtils.createUserPayloadFromConf(jobConf));

    InputSpec reduceInputSpec =
        new InputSpec(
            mapVertexName,
            new InputDescriptor(LocalMergedInput.class.getName())
                .setUserPayload(TezUtils.createUserPayloadFromConf(jobConf)),
            1);
    OutputSpec reduceOutputSpec =
        new OutputSpec(
            "NullDestinationVertex",
            new OutputDescriptor(MROutputLegacy.class.getName())
                .setUserPayload(TezUtils.createUserPayloadFromConf(jobConf)),
            1);

    // Now run a reduce
    TaskSpec taskSpec =
        new TaskSpec(
            TezTestUtils.getMockTaskAttemptId(0, 1, 0, 0),
            dagName,
            reduceVertexName,
            reduceProcessorDesc,
            Collections.singletonList(reduceInputSpec),
            Collections.singletonList(reduceOutputSpec),
            null);

    Map<String, ByteBuffer> serviceConsumerMetadata = new HashMap<String, ByteBuffer>();
    serviceConsumerMetadata.put(
        ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ShuffleUtils.convertJobTokenToBytes(shuffleToken));

    LogicalIOProcessorRuntimeTask task =
        new LogicalIOProcessorRuntimeTask(
            taskSpec,
            0,
            jobConf,
            new String[] {workDir.toString()},
            new TestUmbilical(),
            serviceConsumerMetadata,
            HashMultimap.<String, String>create());

    task.initialize();
    task.run();
    task.close();

    // MRTask mrTask = (MRTask)t.getProcessor();
    // TODO NEWTEZ Verify the partitioner has not been created
    // Likely not applicable anymore.
    // Assert.assertNull(mrTask.getPartitioner());

    // Only a task commit happens, hence the data is still in the temporary directory.
    Path reduceOutputDir =
        new Path(
            new Path(workDir, "output"),
            "_temporary/0/" + IDConverter.toMRTaskIdForOutput(TezTestUtils.getMockTaskId(0, 1, 0)));

    Path reduceOutputFile = new Path(reduceOutputDir, "part-v001-o000-00000");

    SequenceFile.Reader reader = new SequenceFile.Reader(localFs, reduceOutputFile, jobConf);

    LongWritable key = new LongWritable();
    Text value = new Text();
    long prev = Long.MIN_VALUE;
    while (reader.next(key, value)) {
      if (prev != Long.MIN_VALUE) {
        Assert.assertTrue(prev < key.get());
        prev = key.get();
      }
    }

    reader.close();
  }