You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ss...@apache.org on 2015/08/06 11:25:53 UTC
[01/51] [abbrv] tez git commit: TEZ-2090. Add tests for jobs running
in external services. (sseth)
Repository: tez
Updated Branches:
refs/heads/TEZ-2003 bd6fcf95d -> 5fdbe04ff (forced update)
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
new file mode 100644
index 0000000..a93c1a4
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -0,0 +1,183 @@
+/*
+ * 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.tez.tests;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.app.launcher.TezTestServiceNoOpContainerLauncher;
+import org.apache.tez.dag.app.rm.TezTestServiceTaskSchedulerService;
+import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
+import org.apache.tez.examples.HashJoinExample;
+import org.apache.tez.examples.JoinDataGen;
+import org.apache.tez.examples.JoinValidate;
+import org.apache.tez.service.MiniTezTestServiceCluster;
+import org.apache.tez.test.MiniTezCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestExternalTezServices {
+
+ private static final Log LOG = LogFactory.getLog(TestExternalTezServices.class);
+
+ private static MiniTezCluster tezCluster;
+ private static MiniDFSCluster dfsCluster;
+ private static MiniTezTestServiceCluster tezTestServiceCluster;
+
+ private static Configuration clusterConf = new Configuration();
+ private static Configuration confForJobs;
+
+ private static FileSystem remoteFs;
+ private static FileSystem localFs;
+
+ private static TezClient sharedTezClient;
+
+ private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestExternalTezServices.class.getName()
+ + "-tmpDir";
+
+ @BeforeClass
+ public static void setup() throws IOException, TezException, InterruptedException {
+
+ localFs = FileSystem.getLocal(clusterConf);
+
+ try {
+ clusterConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
+ dfsCluster =
+ new MiniDFSCluster.Builder(clusterConf).numDataNodes(1).format(true).racks(null).build();
+ remoteFs = dfsCluster.getFileSystem();
+ LOG.info("MiniDFSCluster started");
+ } catch (IOException io) {
+ throw new RuntimeException("problem starting mini dfs cluster", io);
+ }
+
+ tezCluster = new MiniTezCluster(TestExternalTezServices.class.getName(), 1, 1, 1);
+ Configuration conf = new Configuration();
+ conf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+ tezCluster.init(conf);
+ tezCluster.start();
+ LOG.info("MiniTezCluster started");
+
+ clusterConf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+ for (Map.Entry<String, String> entry : tezCluster.getConfig()) {
+ clusterConf.set(entry.getKey(), entry.getValue());
+ }
+ long jvmMax = Runtime.getRuntime().maxMemory();
+
+ tezTestServiceCluster = MiniTezTestServiceCluster
+ .create(TestExternalTezServices.class.getSimpleName(), 3, ((long) (jvmMax * 0.5d)), 1);
+ tezTestServiceCluster.init(clusterConf);
+ tezTestServiceCluster.start();
+ LOG.info("MiniTezTestServer started");
+
+ confForJobs = new Configuration(clusterConf);
+ for (Map.Entry<String, String> entry : tezTestServiceCluster
+ .getClusterSpecificConfiguration()) {
+ confForJobs.set(entry.getKey(), entry.getValue());
+ }
+
+ // TODO TEZ-2003 Once per vertex configuration is possible, run separate tests for push vs pull (regular threaded execution)
+
+ Path stagingDirPath = new Path("/tmp/tez-staging-dir");
+ remoteFs.mkdirs(stagingDirPath);
+ // This is currently configured to push tasks into the Service, and then use the standard RPC
+ confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
+ confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS,
+ TezTestServiceTaskSchedulerService.class.getName());
+ confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS,
+ TezTestServiceNoOpContainerLauncher.class.getName());
+ confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS,
+ TezTestServiceTaskCommunicatorImpl.class.getName());
+
+ TezConfiguration tezConf = new TezConfiguration(confForJobs);
+
+ sharedTezClient = TezClient.create(TestExternalTezServices.class.getSimpleName() + "_session",
+ tezConf, true);
+ sharedTezClient.start();
+ LOG.info("Shared TezSession started");
+ sharedTezClient.waitTillReady();
+ LOG.info("Shared TezSession ready for submission");
+
+ }
+
+ @AfterClass
+ public static void tearDown() throws IOException, TezException {
+ if (sharedTezClient != null) {
+ sharedTezClient.stop();
+ sharedTezClient = null;
+ }
+
+ if (tezTestServiceCluster != null) {
+ tezTestServiceCluster.stop();
+ tezTestServiceCluster = null;
+ }
+
+ if (tezCluster != null) {
+ tezCluster.stop();
+ tezCluster = null;
+ }
+ if (dfsCluster != null) {
+ dfsCluster.shutdown();
+ dfsCluster = null;
+ }
+ // TODO Add cleanup code.
+ }
+
+
+ @Test(timeout = 60000)
+ public void test1() throws Exception {
+ Path testDir = new Path("/tmp/testHashJoinExample");
+
+ remoteFs.mkdirs(testDir);
+
+ Path dataPath1 = new Path(testDir, "inPath1");
+ Path dataPath2 = new Path(testDir, "inPath2");
+ Path expectedOutputPath = new Path(testDir, "expectedOutputPath");
+ Path outPath = new Path(testDir, "outPath");
+
+ TezConfiguration tezConf = new TezConfiguration(confForJobs);
+
+ JoinDataGen dataGen = new JoinDataGen();
+ String[] dataGenArgs = new String[]{
+ dataPath1.toString(), "1048576", dataPath2.toString(), "524288",
+ expectedOutputPath.toString(), "2"};
+ assertEquals(0, dataGen.run(tezConf, dataGenArgs, sharedTezClient));
+
+ HashJoinExample joinExample = new HashJoinExample();
+ String[] args = new String[]{
+ dataPath1.toString(), dataPath2.toString(), "2", outPath.toString()};
+ assertEquals(0, joinExample.run(tezConf, args, sharedTezClient));
+
+ JoinValidate joinValidate = new JoinValidate();
+ String[] validateArgs = new String[]{
+ expectedOutputPath.toString(), outPath.toString(), "3"};
+ assertEquals(0, joinValidate.run(tezConf, validateArgs, sharedTezClient));
+
+ // Ensure this was actually submitted to the external cluster
+ assertTrue(tezTestServiceCluster.getNumSubmissions() > 0);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java b/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
new file mode 100644
index 0000000..60ebc53
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
@@ -0,0 +1,172 @@
+/*
+ * 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.tez.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.GroupInputSpec;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.GroupInputSpecProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.IOSpecProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
+
+public class ProtoConverters {
+
+ public static TaskSpec getTaskSpecfromProto(TaskSpecProto taskSpecProto) {
+ TezTaskAttemptID taskAttemptID =
+ TezTaskAttemptID.fromString(taskSpecProto.getTaskAttemptIdString());
+
+ ProcessorDescriptor processorDescriptor = null;
+ if (taskSpecProto.hasProcessorDescriptor()) {
+ processorDescriptor = DagTypeConverters
+ .convertProcessorDescriptorFromDAGPlan(taskSpecProto.getProcessorDescriptor());
+ }
+
+ List<InputSpec> inputSpecList = new ArrayList<InputSpec>(taskSpecProto.getInputSpecsCount());
+ if (taskSpecProto.getInputSpecsCount() > 0) {
+ for (IOSpecProto inputSpecProto : taskSpecProto.getInputSpecsList()) {
+ inputSpecList.add(getInputSpecFromProto(inputSpecProto));
+ }
+ }
+
+ List<OutputSpec> outputSpecList =
+ new ArrayList<OutputSpec>(taskSpecProto.getOutputSpecsCount());
+ if (taskSpecProto.getOutputSpecsCount() > 0) {
+ for (IOSpecProto outputSpecProto : taskSpecProto.getOutputSpecsList()) {
+ outputSpecList.add(getOutputSpecFromProto(outputSpecProto));
+ }
+ }
+
+ List<GroupInputSpec> groupInputSpecs =
+ new ArrayList<GroupInputSpec>(taskSpecProto.getGroupedInputSpecsCount());
+ if (taskSpecProto.getGroupedInputSpecsCount() > 0) {
+ for (GroupInputSpecProto groupInputSpecProto : taskSpecProto.getGroupedInputSpecsList()) {
+ groupInputSpecs.add(getGroupInputSpecFromProto(groupInputSpecProto));
+ }
+ }
+
+ TaskSpec taskSpec =
+ new TaskSpec(taskAttemptID, taskSpecProto.getDagName(), taskSpecProto.getVertexName(),
+ taskSpecProto.getVertexParallelism(), processorDescriptor, inputSpecList,
+ outputSpecList, groupInputSpecs);
+ return taskSpec;
+ }
+
+ public static TaskSpecProto convertTaskSpecToProto(TaskSpec taskSpec) {
+ TaskSpecProto.Builder builder = TaskSpecProto.newBuilder();
+ builder.setTaskAttemptIdString(taskSpec.getTaskAttemptID().toString());
+ builder.setDagName(taskSpec.getDAGName());
+ builder.setVertexName(taskSpec.getVertexName());
+ builder.setVertexParallelism(taskSpec.getVertexParallelism());
+
+ if (taskSpec.getProcessorDescriptor() != null) {
+ builder.setProcessorDescriptor(
+ DagTypeConverters.convertToDAGPlan(taskSpec.getProcessorDescriptor()));
+ }
+
+ if (taskSpec.getInputs() != null && !taskSpec.getInputs().isEmpty()) {
+ for (InputSpec inputSpec : taskSpec.getInputs()) {
+ builder.addInputSpecs(convertInputSpecToProto(inputSpec));
+ }
+ }
+
+ if (taskSpec.getOutputs() != null && !taskSpec.getOutputs().isEmpty()) {
+ for (OutputSpec outputSpec : taskSpec.getOutputs()) {
+ builder.addOutputSpecs(convertOutputSpecToProto(outputSpec));
+ }
+ }
+
+ if (taskSpec.getGroupInputs() != null && !taskSpec.getGroupInputs().isEmpty()) {
+ for (GroupInputSpec groupInputSpec : taskSpec.getGroupInputs()) {
+ builder.addGroupedInputSpecs(convertGroupInputSpecToProto(groupInputSpec));
+
+ }
+ }
+ return builder.build();
+ }
+
+
+ public static InputSpec getInputSpecFromProto(IOSpecProto inputSpecProto) {
+ InputDescriptor inputDescriptor = null;
+ if (inputSpecProto.hasIoDescriptor()) {
+ inputDescriptor =
+ DagTypeConverters.convertInputDescriptorFromDAGPlan(inputSpecProto.getIoDescriptor());
+ }
+ InputSpec inputSpec = new InputSpec(inputSpecProto.getConnectedVertexName(), inputDescriptor,
+ inputSpecProto.getPhysicalEdgeCount());
+ return inputSpec;
+ }
+
+ public static IOSpecProto convertInputSpecToProto(InputSpec inputSpec) {
+ IOSpecProto.Builder builder = IOSpecProto.newBuilder();
+ if (inputSpec.getSourceVertexName() != null) {
+ builder.setConnectedVertexName(inputSpec.getSourceVertexName());
+ }
+ if (inputSpec.getInputDescriptor() != null) {
+ builder.setIoDescriptor(DagTypeConverters.convertToDAGPlan(inputSpec.getInputDescriptor()));
+ }
+ builder.setPhysicalEdgeCount(inputSpec.getPhysicalEdgeCount());
+ return builder.build();
+ }
+
+ public static OutputSpec getOutputSpecFromProto(IOSpecProto outputSpecProto) {
+ OutputDescriptor outputDescriptor = null;
+ if (outputSpecProto.hasIoDescriptor()) {
+ outputDescriptor =
+ DagTypeConverters.convertOutputDescriptorFromDAGPlan(outputSpecProto.getIoDescriptor());
+ }
+ OutputSpec outputSpec =
+ new OutputSpec(outputSpecProto.getConnectedVertexName(), outputDescriptor,
+ outputSpecProto.getPhysicalEdgeCount());
+ return outputSpec;
+ }
+
+ public static IOSpecProto convertOutputSpecToProto(OutputSpec outputSpec) {
+ IOSpecProto.Builder builder = IOSpecProto.newBuilder();
+ if (outputSpec.getDestinationVertexName() != null) {
+ builder.setConnectedVertexName(outputSpec.getDestinationVertexName());
+ }
+ if (outputSpec.getOutputDescriptor() != null) {
+ builder.setIoDescriptor(DagTypeConverters.convertToDAGPlan(outputSpec.getOutputDescriptor()));
+ }
+ builder.setPhysicalEdgeCount(outputSpec.getPhysicalEdgeCount());
+ return builder.build();
+ }
+
+ public static GroupInputSpec getGroupInputSpecFromProto(GroupInputSpecProto groupInputSpecProto) {
+ GroupInputSpec groupSpec = new GroupInputSpec(groupInputSpecProto.getGroupName(),
+ groupInputSpecProto.getGroupVerticesList(), DagTypeConverters
+ .convertInputDescriptorFromDAGPlan(groupInputSpecProto.getMergedInputDescriptor()));
+ return groupSpec;
+ }
+
+ public static GroupInputSpecProto convertGroupInputSpecToProto(GroupInputSpec groupInputSpec) {
+ GroupInputSpecProto.Builder builder = GroupInputSpecProto.newBuilder();
+ builder.setGroupName(groupInputSpec.getGroupName());
+ builder.addAllGroupVertices(groupInputSpec.getGroupVertices());
+ builder.setMergedInputDescriptor(
+ DagTypeConverters.convertToDAGPlan(groupInputSpec.getMergedInputDescriptor()));
+ return builder.build();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto b/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
new file mode 100644
index 0000000..2f8b2e6
--- /dev/null
+++ b/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+option java_package = "org.apache.tez.test.service.rpc";
+option java_outer_classname = "TezTestServiceProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "DAGApiRecords.proto";
+
+message IOSpecProto {
+ optional string connected_vertex_name = 1;
+ optional TezEntityDescriptorProto io_descriptor = 2;
+ optional int32 physical_edge_count = 3;
+}
+
+message GroupInputSpecProto {
+ optional string group_name = 1;
+ repeated string group_vertices = 2;
+ optional TezEntityDescriptorProto merged_input_descriptor = 3;
+}
+
+message TaskSpecProto {
+ optional string task_attempt_id_string = 1;
+ optional string dag_name = 2;
+ optional string vertex_name = 3;
+ optional TezEntityDescriptorProto processor_descriptor = 4;
+ repeated IOSpecProto input_specs = 5;
+ repeated IOSpecProto output_specs = 6;
+ repeated GroupInputSpecProto grouped_input_specs = 7;
+ optional int32 vertex_parallelism = 8;
+}
+
+
+message SubmitWorkRequestProto {
+ optional string container_id_string = 1;
+ optional string am_host = 2;
+ optional int32 am_port = 3;
+ optional string token_identifier = 4;
+ optional bytes credentials_binary = 5;
+ optional string user = 6;
+ optional string application_id_string = 7;
+ optional int32 app_attempt_number = 8;
+ optional TaskSpecProto task_spec = 9;
+}
+
+message SubmitWorkResponseProto {
+}
+
+
+
+message RunContainerRequestProto {
+ optional string container_id_string = 1;
+ optional string am_host = 2;
+ optional int32 am_port = 3;
+ optional string token_identifier = 4;
+ optional bytes credentials_binary = 5;
+ optional string user = 6;
+ optional string application_id_string = 7;
+ optional int32 app_attempt_number = 8;
+}
+
+message RunContainerResponseProto {
+}
+
+service TezTestServiceProtocol {
+ rpc runContainer(RunContainerRequestProto) returns (RunContainerResponseProto);
+ rpc submitWork(SubmitWorkRequestProto) returns (SubmitWorkResponseProto);
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/resources/log4j.properties b/tez-ext-service-tests/src/test/resources/log4j.properties
new file mode 100644
index 0000000..531b68b
--- /dev/null
+++ b/tez-ext-service-tests/src/test/resources/log4j.properties
@@ -0,0 +1,19 @@
+# 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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index 94efbee..e6ef5e2 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -404,7 +404,7 @@ public class TezChild {
private final Throwable throwable;
private final String errorMessage;
- ContainerExecutionResult(ExitStatus exitStatus, @Nullable Throwable throwable,
+ public ContainerExecutionResult(ExitStatus exitStatus, @Nullable Throwable throwable,
@Nullable String errorMessage) {
this.exitStatus = exitStatus;
this.throwable = throwable;
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index de83889..f54814b 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -67,7 +67,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
private final AtomicBoolean taskRunning;
private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
- TezTaskRunner(Configuration tezConf, UserGroupInformation ugi, String[] localDirs,
+ public TezTaskRunner(Configuration tezConf, UserGroupInformation ugi, String[] localDirs,
TaskSpec taskSpec, int appAttemptNumber,
Map<String, ByteBuffer> serviceConsumerMetadata, Map<String, String> serviceProviderEnvMap,
Multimap<String, String> startedInputsMap, TaskReporter taskReporter,
[31/51] [abbrv] tez git commit: TEZ-2502. Fix for TezTaskRunner2 not
killing tasks properly in all situations. (sseth)
Posted by ss...@apache.org.
TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/c5d0062e
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/c5d0062e
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/c5d0062e
Branch: refs/heads/TEZ-2003
Commit: c5d0062e9bbda20b5986e34072b0b15cab64337e
Parents: b41dbb0
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu May 28 18:29:12 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../apache/tez/runtime/task/TezTaskRunner2.java | 83 ++++++++++++--------
2 files changed, 53 insertions(+), 31 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/c5d0062e/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index e333832..42c2e1e 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -29,5 +29,6 @@ ALL CHANGES:
TEZ-2465. Retrun the status of a kill request in TaskRunner2.
TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
+ TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/c5d0062e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index 15629fd..a5fabb5 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -124,6 +124,8 @@ public class TezTaskRunner2 {
try {
ListenableFuture<TaskRunner2CallableResult> future = null;
synchronized (this) {
+ // All running state changes must be made within a synchronized block to ensure
+ // kills are issued or the task is not setup.
if (isRunningState()) {
// Safe to do this within a synchronized block because we're providing
// the handler on which the Reporter will communicate back. Assuming
@@ -252,27 +254,34 @@ public class TezTaskRunner2 {
* @return true if the task kill was honored, false otherwise
*/
public boolean killTask() {
+ boolean isFirstError = false;
synchronized (this) {
if (isRunningState()) {
if (trySettingEndReason(EndReason.KILL_REQUESTED)) {
+ isFirstError = true;
killTaskRequested.set(true);
- if (taskRunnerCallable != null) {
- taskKillStartTime = System.currentTimeMillis();
- taskRunnerCallable.interruptTask();
- }
- return true;
} else {
- LOG.info("Ignoring killTask request for {} since end reason is already set to {}",
- task.getTaskAttemptID(), firstEndReason);
+ logErrorIngored("killTask", null);
}
} else {
- LOG.info("Ignoring killTask request for {} since it is not in a running state",
- task.getTaskAttemptID());
+ logErrorIngored("killTask", null);
}
}
- return false;
+ if (isFirstError) {
+ logAborting("killTask");
+ killTaskInternal();
+ return true;
+ } else {
+ return false;
+ }
}
+ private void killTaskInternal() {
+ if (taskRunnerCallable != null) {
+ taskKillStartTime = System.currentTimeMillis();
+ taskRunnerCallable.interruptTask();
+ }
+ }
// Checks and changes on these states should happen within a synchronized block,
// to ensure the first event is the one that is captured and causes specific behaviour.
@@ -310,17 +319,18 @@ public class TezTaskRunner2 {
errorReporterToAm.set(true);
oobSignalErrorInProgress = true;
} else {
- LOG.info(
- "Ignoring fatal error since the task has ended for reason: {}. IgnoredError: {} ",
- firstEndReason, (t == null ? message : t.getMessage()));
+ logErrorIngored("signalFatalError", message);
}
+ } else {
+ logErrorIngored("signalFatalError", message);
}
}
// Informing the TaskReporter here because the running task may not be interruptable.
// Has to be outside the lock.
if (isFirstError) {
- killTask();
+ logAborting("signalFatalError");
+ killTaskInternal();
try {
taskReporter.taskFailed(taskAttemptID, t, getTaskDiagnosticsString(t, message), sourceInfo);
} catch (IOException e) {
@@ -371,19 +381,22 @@ public class TezTaskRunner2 {
if (trySettingEndReason(EndReason.COMMUNICATION_FAILURE)) {
registerFirstException(t, null);
isFirstError = true;
+ } else {
+ logErrorIngored("umbilicalFatalError", null);
}
// A race is possible between a task succeeding, and a subsequent timed heartbeat failing.
// These errors can be ignored, since a task can only succeed if the synchronous taskSucceeded
// method does not throw an exception, in which case task success is registered with the AM.
// Leave subsequent heartbeat errors to the next entity to communicate using the TaskReporter
} else {
- LOG.info("Ignoring Communication failure since task with id=" + task.getTaskAttemptID()
- + " is already complete, is failing or has been asked to terminate");
+ logErrorIngored("umbilicalFatalError", null);
}
+ // Since this error came from the taskReporter - there's no point attempting to report a failure back to it.
+ // However, the task does need to be cleaned up
}
- // Since this error came from the taskReporter - there's no point attempting to report a failure back to it.
if (isFirstError) {
- killTask();
+ logAborting("umbilicalFatalError");
+ killTaskInternal();
}
}
@@ -395,18 +408,12 @@ public class TezTaskRunner2 {
isFirstTerminate = trySettingEndReason(EndReason.CONTAINER_STOP_REQUESTED);
// Respect stopContainerRequested since it can come in at any point, despite a previous failure.
stopContainerRequested.set(true);
-
- if (isFirstTerminate) {
- LOG.info("Attempting to abort {} since a shutdown request was received",
- task.getTaskAttemptID());
- if (taskRunnerCallable != null) {
- taskKillStartTime = System.currentTimeMillis();
- taskRunnerCallable.interruptTask();
- }
- } else {
- LOG.info("Not acting on shutdown request for {} since the task is not in running state",
- task.getTaskAttemptID());
- }
+ }
+ if (isFirstTerminate) {
+ logAborting("shutdownRequested");
+ killTaskInternal();
+ } else {
+ logErrorIngored("shutdownRequested", null);
}
}
}
@@ -451,6 +458,20 @@ public class TezTaskRunner2 {
private void handleFinalStatusUpdateFailure(Throwable t, boolean successReportAttempted) {
// TODO Ideally differentiate between FAILED/KILLED
- LOG.warn("Failure while reporting state= {} to AM", (successReportAttempted ? "success" : "failure/killed"), t);
+ LOG.warn("Failure while reporting state= {} to AM",
+ (successReportAttempted ? "success" : "failure/killed"), t);
+ }
+
+ private void logErrorIngored(String ignoredEndReason, String errorMessage) {
+ LOG.info(
+ "Ignoring {} request since the task with id {} has ended for reason: {}. IgnoredError: {} ",
+ ignoredEndReason, task.getTaskAttemptID(),
+ firstEndReason, (firstException == null ? (errorMessage == null ? "" : errorMessage) :
+ firstException.getMessage()));
+ }
+
+ private void logAborting(String abortReason) {
+ LOG.info("Attempting to abort {} due to an invocation of {}", task.getTaskAttemptID(),
+ abortReason);
}
}
\ No newline at end of file
[39/51] [abbrv] tez git commit: TEZ-2005. Define basic interface for
pluggable TaskScheduler. (sseth)
Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
index d4cf317..1e76dc9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
@@ -37,6 +37,10 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AMState;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.math3.random.RandomDataGenerator;
@@ -59,10 +63,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.DAGAppMasterState;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.common.ContainerSignatureMatcher;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@@ -80,17 +81,14 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
eventHandler.handle(new AMNodeEventNodeCountUpdated(clusterNmCount));
}
*/
-public class YarnTaskSchedulerService extends TaskSchedulerService
+public class YarnTaskSchedulerService extends TaskScheduler
implements AMRMClientAsync.CallbackHandler {
private static final Logger LOG = LoggerFactory.getLogger(YarnTaskSchedulerService.class);
final TezAMRMClientAsync<CookieContainerRequest> amRmClient;
- final TaskSchedulerAppCallback realAppClient;
- final TaskSchedulerAppCallback appClientDelegate;
final ContainerSignatureMatcher containerSignatureMatcher;
- ExecutorService appCallbackExecutor;
// Container Re-Use configuration
private boolean shouldReuseContainers;
@@ -131,7 +129,6 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
final String appHostName;
final int appHostPort;
final String appTrackingUrl;
- final AppContext appContext;
private AtomicBoolean hasUnregistered = new AtomicBoolean(false);
AtomicBoolean isStopped = new AtomicBoolean(false);
@@ -150,6 +147,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
Set<ContainerId> sessionMinHeldContainers = Sets.newHashSet();
RandomDataGenerator random = new RandomDataGenerator();
+ private final Configuration conf;
@VisibleForTesting
protected AtomicBoolean shouldUnregister = new AtomicBoolean(false);
@@ -213,51 +211,29 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
}
}
- public YarnTaskSchedulerService(TaskSchedulerAppCallback appClient,
- ContainerSignatureMatcher containerSignatureMatcher,
- String appHostName,
- int appHostPort,
- String appTrackingUrl,
- AppContext appContext) {
- super(YarnTaskSchedulerService.class.getName());
- this.realAppClient = appClient;
- this.appCallbackExecutor = createAppCallbackExecutorService();
- this.containerSignatureMatcher = containerSignatureMatcher;
- this.appClientDelegate = createAppCallbackDelegate(appClient);
+ public YarnTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) {
+ super(taskSchedulerContext);
+ this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher();
this.amRmClient = TezAMRMClientAsync.createAMRMClientAsync(1000, this);
- this.appHostName = appHostName;
- this.appHostPort = appHostPort;
- this.appTrackingUrl = appTrackingUrl;
- this.appContext = appContext;
+ this.appHostName = taskSchedulerContext.getAppHostName();
+ this.appHostPort = taskSchedulerContext.getAppClientPort();
+ this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
+ this.conf = taskSchedulerContext.getInitialConfiguration();
}
@Private
@VisibleForTesting
- YarnTaskSchedulerService(TaskSchedulerAppCallback appClient,
- ContainerSignatureMatcher containerSignatureMatcher,
- String appHostName,
- int appHostPort,
- String appTrackingUrl,
- TezAMRMClientAsync<CookieContainerRequest> client,
- AppContext appContext) {
- super(YarnTaskSchedulerService.class.getName());
- this.realAppClient = appClient;
- this.appCallbackExecutor = createAppCallbackExecutorService();
- this.containerSignatureMatcher = containerSignatureMatcher;
- this.appClientDelegate = createAppCallbackDelegate(appClient);
+ YarnTaskSchedulerService(TaskSchedulerContext taskSchedulerContext,
+ TezAMRMClientAsync<CookieContainerRequest> client) {
+ super(taskSchedulerContext);
+ this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher();
this.amRmClient = client;
- this.appHostName = appHostName;
- this.appHostPort = appHostPort;
- this.appTrackingUrl = appTrackingUrl;
- this.appContext = appContext;
+ this.appHostName = taskSchedulerContext.getAppHostName();
+ this.appHostPort = taskSchedulerContext.getAppClientPort();
+ this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
+ this.conf = taskSchedulerContext.getInitialConfiguration();
}
- @VisibleForTesting
- ExecutorService createAppCallbackExecutorService() {
- return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
- .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
- }
-
@Override
public Resource getAvailableResources() {
return amRmClient.getAvailableResources();
@@ -269,12 +245,6 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
return amRmClient.getClusterNodeCount();
}
- TaskSchedulerAppCallback createAppCallbackDelegate(
- TaskSchedulerAppCallback realAppClient) {
- return new TaskSchedulerAppCallbackWrapper(realAppClient,
- appCallbackExecutor);
- }
-
@Override
public void setShouldUnregister() {
this.shouldUnregister.set(true);
@@ -287,8 +257,9 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
// AbstractService methods
@Override
- public synchronized void serviceInit(Configuration conf) {
+ public synchronized void initialize() {
+ // TODO Post TEZ-2003. Make all of these final fields.
amRmClient.init(conf);
int heartbeatIntervalMax = conf.getInt(
TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX,
@@ -361,7 +332,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
}
@Override
- public void serviceStart() {
+ public void start() {
try {
RegisterApplicationMasterResponse response;
synchronized (this) {
@@ -371,7 +342,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
appTrackingUrl);
}
// upcall to app outside locks
- appClientDelegate.setApplicationRegistrationData(
+ getContext().setApplicationRegistrationData(
response.getMaximumResourceCapability(),
response.getApplicationACLs(),
response.getClientToAMTokenMasterKey());
@@ -387,7 +358,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
}
@Override
- public void serviceStop() throws InterruptedException {
+ public void shutdown() throws InterruptedException {
// upcall to app outside of locks
try {
delayedContainerManager.shutdown();
@@ -396,7 +367,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
synchronized (this) {
isStopped.set(true);
if (shouldUnregister.get()) {
- AppFinalStatus status = appClientDelegate.getFinalAppStatus();
+ AppFinalStatus status = getContext().getFinalAppStatus();
LOG.info("Unregistering application from RM"
+ ", exitStatus=" + status.exitStatus
+ ", exitMessage=" + status.exitMessage
@@ -413,8 +384,6 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
// operation and at the same time the callback operation might be trying
// to get our lock.
amRmClient.stop();
- appCallbackExecutor.shutdown();
- appCallbackExecutor.awaitTermination(1000l, TimeUnit.MILLISECONDS);
} catch (YarnException e) {
LOG.error("Yarn Exception while unregistering ", e);
throw new TezUncheckedException(e);
@@ -478,7 +447,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
// upcall to app must be outside locks
for (Entry<Object, ContainerStatus> entry : appContainerStatus.entrySet()) {
- appClientDelegate.containerCompleted(entry.getKey(), entry.getValue());
+ getContext().containerCompleted(entry.getKey(), entry.getValue());
}
}
@@ -528,7 +497,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
private synchronized Map<CookieContainerRequest, Container>
assignNewlyAllocatedContainers(Iterable<Container> containers) {
- boolean amInCompletionState = appContext.isAMInCompletionState();
+ boolean amInCompletionState = (getContext().getAMState() == AMState.COMPLETED);
Map<CookieContainerRequest, Container> assignedContainers =
new HashMap<CookieContainerRequest, Container>();
@@ -550,7 +519,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
private synchronized Map<CookieContainerRequest, Container>
tryAssignReUsedContainers(Iterable<Container> containers) {
- boolean amInCompletionState = appContext.isAMInCompletionState();
+ boolean amInCompletionState = (getContext().getAMState() == AMState.COMPLETED);
Map<CookieContainerRequest, Container> assignedContainers =
new HashMap<CookieContainerRequest, Container>();
@@ -590,7 +559,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
private synchronized Map<CookieContainerRequest, Container>
assignDelayedContainer(HeldContainer heldContainer) {
- DAGAppMasterState state = appContext.getAMState();
+ AMState state = getContext().getAMState();
boolean isNew = heldContainer.isNew();
if (LOG.isDebugEnabled()) {
@@ -606,13 +575,13 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
+ ", isNew=" + isNew);
}
- if (state.equals(DAGAppMasterState.IDLE) || taskRequests.isEmpty()) {
+ if (state.equals(AMState.IDLE) || taskRequests.isEmpty()) {
// reset locality level on held container
// if sessionDelay defined, push back into delayed queue if not already
// done so
// Compute min held containers.
- if (appContext.isSession() && sessionNumMinHeldContainers > 0 &&
+ if (getContext().isSession() && sessionNumMinHeldContainers > 0 &&
sessionMinHeldContainers.isEmpty()) {
// session mode and need to hold onto containers and not done so already
determineMinHeldContainers();
@@ -626,7 +595,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
&& idleContainerTimeoutMin != -1)) {
// container idle timeout has expired or is a new unused container.
// new container is possibly a spurious race condition allocation.
- if (appContext.isSession()
+ if (getContext().isSession()
&& sessionMinHeldContainers.contains(heldContainer.getContainer().getId())) {
// There are no outstanding requests. So its safe to hold new containers.
// We may have received more containers than necessary and some are unused
@@ -667,7 +636,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
heldContainer.getContainer(), currentTime
+ localitySchedulingDelay);
}
- } else if (state.equals(DAGAppMasterState.RUNNING)) {
+ } else if (state.equals(AMState.RUNNING_APP)) {
// clear min held containers since we need to allocate to tasks
if (!sessionMinHeldContainers.isEmpty()) {
// update the expire time of min held containers so that they are
@@ -806,12 +775,12 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
// Are there any pending requests at any priority?
// release if there are tasks or this is not a session
if (safeToRelease &&
- (!taskRequests.isEmpty() || !appContext.isSession())) {
+ (!taskRequests.isEmpty() || !getContext().isSession())) {
LOG.info("Releasing held container as either there are pending but "
+ " unmatched requests or this is not a session"
+ ", containerId=" + heldContainer.container.getId()
+ ", pendingTasks=" + taskRequests.size()
- + ", isSession=" + appContext.isSession()
+ + ", isSession=" + getContext().isSession()
+ ". isNew=" + isNew);
releaseUnassignedContainers(
Lists.newArrayList(heldContainer.container));
@@ -862,7 +831,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
return;
}
// upcall to app must be outside locks
- appClientDelegate.appShutdownRequested();
+ getContext().appShutdownRequested();
}
@Override
@@ -872,7 +841,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
}
// ignore bad nodes for now
// upcall to app must be outside locks
- appClientDelegate.nodesUpdated(updatedNodes);
+ getContext().nodesUpdated(updatedNodes);
}
@Override
@@ -894,7 +863,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
numHeartbeats++;
preemptIfNeeded();
- return appClientDelegate.getProgress();
+ return getContext().getProgress();
}
@Override
@@ -902,7 +871,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
if (isStopped.get()) {
return;
}
- appClientDelegate.onError(t);
+ getContext().onError(t);
}
@Override
@@ -1289,7 +1258,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
ContainerId cId = preemptedContainers[i];
if (cId != null) {
LOG.info("Preempting container: " + cId + " currently allocated to a task.");
- appClientDelegate.preemptContainer(cId);
+ getContext().preemptContainer(cId);
}
}
}
@@ -1422,7 +1391,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
Object assignedTask = containerAssignments.remove(containerId);
if (assignedTask != null) {
// A task was assigned to this container at some point. Inform the app.
- appClientDelegate.containerBeingReleased(containerId);
+ getContext().containerBeingReleased(containerId);
}
HeldContainer delayedContainer = heldContainers.remove(containerId);
if (delayedContainer != null) {
@@ -1626,7 +1595,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
private void informAppAboutAssignment(CookieContainerRequest assigned,
Container container) {
- appClientDelegate.taskAllocated(getTask(assigned),
+ getContext().taskAllocated(getTask(assigned),
assigned.getCookie().getAppCookie(), container);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index e63d86d..02f5026 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -29,6 +29,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.common.ContainerSignatureMatcher;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
index 938096d..fcb9eaf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
@@ -22,6 +22,7 @@ import java.util.Collection;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.common.ContainerSignatureMatcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.service.AbstractService;
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java
index 211c537..436f098 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerContextMatcher.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.tez.dag.app.ContainerContext;
import com.google.common.base.Preconditions;
+import org.apache.tez.common.ContainerSignatureMatcher;
public class ContainerContextMatcher implements ContainerSignatureMatcher {
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerSignatureMatcher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerSignatureMatcher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerSignatureMatcher.java
deleted file mode 100644
index 0f9c2d6..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/ContainerSignatureMatcher.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/* Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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.tez.dag.app.rm.container;
-
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.LocalResource;
-
-public interface ContainerSignatureMatcher {
- /**
- * Checks the compatibility between the specified container signatures.
- *
- * @return true if the first signature is a super set of the second
- * signature.
- */
- public boolean isSuperSet(Object cs1, Object cs2);
-
- /**
- * Checks if the container signatures match exactly
- * @return true if exact match
- */
- public boolean isExactMatch(Object cs1, Object cs2);
-
- /**
- * Gets additional resources specified in lr2, which are not present for lr1
- *
- * @param lr1
- * @param lr2
- * @return additional resources specified in lr2, which are not present for lr1
- */
- public Map<String, LocalResource> getAdditionalResources(Map<String, LocalResource> lr1,
- Map<String, LocalResource> lr2);
-
-
- /**
- * Do a union of 2 signatures
- * Pre-condition. This function should only be invoked iff cs1 is compatible with cs2.
- * i.e. isSuperSet should not return false.
- * @param cs1 Signature 1 Original signature
- * @param cs2 Signature 2 New signature
- * @return Union of 2 signatures
- */
- public Object union(Object cs1, Object cs2);
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index e37ab4a..88f6066 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isNull;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
@@ -46,7 +45,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
@@ -68,16 +66,14 @@ import org.apache.tez.dag.app.ContainerHeartbeatHandler;
import org.apache.tez.dag.app.DAGAppMasterState;
import org.apache.tez.dag.app.TaskAttemptListener;
import org.apache.tez.dag.app.dag.TaskAttempt;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AMRMClientAsyncForTest;
import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AMRMClientForTest;
import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AlwaysMatchesContainerMatcher;
import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.CapturingEventHandler;
-import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerAppCallbackDrainable;
+import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerContextDrainable;
import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerEventHandlerForTest;
-import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableAppCallback;
+import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableContext;
import org.apache.tez.dag.app.rm.container.AMContainerEventAssignTA;
import org.apache.tez.dag.app.rm.container.AMContainerEventStopRequest;
import org.apache.tez.dag.app.rm.container.AMContainerMap;
@@ -116,14 +112,13 @@ public class TestContainerReuse {
conf.setBoolean(
TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, true);
conf.setBoolean(
- TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false);
+ TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED, false);
conf.setBoolean(
- TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false);
+ TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED, false);
conf.setLong(
TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 3000l);
conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
@@ -132,12 +127,6 @@ public class TestContainerReuse {
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient =
spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(conf).when(appContext).getAMConf();
@@ -161,11 +150,11 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(conf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler =
- (TaskSchedulerWithDrainableAppCallback)
+ TaskSchedulerWithDrainableContext taskScheduler =
+ (TaskSchedulerWithDrainableContext)
((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback =
+ TaskSchedulerContextDrainable drainableAppCallback =
taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
@@ -251,8 +240,7 @@ public class TestContainerReuse {
}
}
assertTrue("containerHost2 was not released", exception == null);
- taskScheduler.stop();
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
@@ -267,7 +255,6 @@ public class TestContainerReuse {
conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 1000l);
conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
@@ -276,12 +263,6 @@ public class TestContainerReuse {
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient =
spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -304,11 +285,11 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(conf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler =
- (TaskSchedulerWithDrainableAppCallback)
+ TaskSchedulerWithDrainableContext taskScheduler =
+ (TaskSchedulerWithDrainableContext)
((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback =
+ TaskSchedulerContextDrainable drainableAppCallback =
taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
@@ -366,8 +347,7 @@ public class TestContainerReuse {
eq(containerHost2.getId()));
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
- taskScheduler.stop();
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
@@ -380,19 +360,12 @@ public class TestContainerReuse {
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0);
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -410,9 +383,9 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler = (TaskSchedulerWithDrainableAppCallback) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
+ TaskSchedulerWithDrainableContext taskScheduler = (TaskSchedulerWithDrainableContext) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+ TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -504,7 +477,7 @@ public class TestContainerReuse {
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
eventHandler.reset();
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
@@ -522,19 +495,11 @@ public class TestContainerReuse {
tezConf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LAUNCH_CMD_OPTS, "dir=/tmp/__VERTEX_NAME__/__TASK_INDEX__");
TaskSpecificLaunchCmdOption taskSpecificLaunchCmdOption = new TaskSpecificLaunchCmdOption(tezConf);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -554,10 +519,10 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler =
- (TaskSchedulerWithDrainableAppCallback) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
+ TaskSchedulerWithDrainableContext taskScheduler =
+ (TaskSchedulerWithDrainableContext) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+ TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -705,7 +670,7 @@ public class TestContainerReuse {
verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta16), any(Object.class), eq(container3));
eventHandler.reset();
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
@@ -721,20 +686,12 @@ public class TestContainerReuse {
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 1000l);
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 1000l);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient =
spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -758,11 +715,11 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler =
- (TaskSchedulerWithDrainableAppCallback)
+ TaskSchedulerWithDrainableContext taskScheduler =
+ (TaskSchedulerWithDrainableContext)
((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback =
+ TaskSchedulerContextDrainable drainableAppCallback =
taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -836,7 +793,7 @@ public class TestContainerReuse {
verify(rmClient).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
@@ -853,20 +810,12 @@ public class TestContainerReuse {
tezConf.setInt(
TezConfiguration.TEZ_AM_SESSION_MIN_HELD_CONTAINERS, 1);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient =
spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -890,11 +839,11 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler =
- (TaskSchedulerWithDrainableAppCallback)
+ TaskSchedulerWithDrainableContext taskScheduler =
+ (TaskSchedulerWithDrainableContext)
((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+ TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -965,7 +914,7 @@ public class TestContainerReuse {
// container should not get released due to min held containers
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
@@ -979,19 +928,11 @@ public class TestContainerReuse {
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0);
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, -1);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID1 = TezDAGID.getInstance("0", 1, 0);
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -1011,9 +952,9 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler = (TaskSchedulerWithDrainableAppCallback) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
+ TaskSchedulerWithDrainableContext taskScheduler = (TaskSchedulerWithDrainableContext) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+ TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -1129,7 +1070,7 @@ public class TestContainerReuse {
assertEquals(2, assignEvent.getRemoteTaskLocalResources().size());
eventHandler.reset();
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
@@ -1143,19 +1084,11 @@ public class TestContainerReuse {
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0);
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, -1);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
-
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID1 = TezDAGID.getInstance("0", 1, 0);
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -1177,9 +1110,9 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler = (TaskSchedulerWithDrainableAppCallback) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
+ TaskSchedulerWithDrainableContext taskScheduler = (TaskSchedulerWithDrainableContext) ((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+ TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -1291,7 +1224,7 @@ public class TestContainerReuse {
verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container2));
eventHandler.reset();
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
@@ -1305,19 +1238,12 @@ public class TestContainerReuse {
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS, 0);
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
tezConf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
CapturingEventHandler eventHandler = new CapturingEventHandler();
TezDAGID dagID = TezDAGID.getInstance("0", 0, 0);
AMRMClient<CookieContainerRequest> rmClientCore = new AMRMClientForTest();
TezAMRMClientAsync<CookieContainerRequest> rmClient = spy(new AMRMClientAsyncForTest(rmClientCore, 100));
- String appUrl = "url";
- String appMsg = "success";
- AppFinalStatus finalStatus =
- new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
-
- doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
doReturn(new Configuration(false)).when(appContext).getAMConf();
@@ -1326,7 +1252,7 @@ public class TestContainerReuse {
AMNodeTracker amNodeTracker = new AMNodeTracker(eventHandler, appContext);
doReturn(amContainerMap).when(appContext).getAllContainers();
doReturn(amNodeTracker).when(appContext).getNodeTracker();
- doReturn(DAGAppMasterState.RUNNING).when(appContext).getAMState();
+ doReturn(DAGAppMasterState.SUCCEEDED).when(appContext).getAMState();
doReturn(true).when(appContext).isAMInCompletionState();
doReturn(dagID).when(appContext).getCurrentDAGID();
doReturn(mock(ClusterInfo.class)).when(appContext).getClusterInfo();
@@ -1338,10 +1264,10 @@ public class TestContainerReuse {
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
- TaskSchedulerWithDrainableAppCallback taskScheduler = (TaskSchedulerWithDrainableAppCallback)
+ TaskSchedulerWithDrainableContext taskScheduler = (TaskSchedulerWithDrainableContext)
((TaskSchedulerEventHandlerForTest) taskSchedulerEventHandler)
.getSpyTaskScheduler();
- TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
+ TaskSchedulerContextDrainable drainableAppCallback = taskScheduler.getDrainableAppCallback();
AtomicBoolean drainNotifier = new AtomicBoolean(false);
taskScheduler.delayedContainerManager.drainedDelayedContainersForTest = drainNotifier;
@@ -1369,7 +1295,7 @@ public class TestContainerReuse {
drainableAppCallback.drain();
verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta11),
any(Object.class), eq(container1));
- taskScheduler.close();
+ taskScheduler.shutdown();
taskSchedulerEventHandler.close();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
index 12390b2..2ada2f1 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
@@ -18,12 +18,12 @@
package org.apache.tez.dag.app.rm;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.concurrent.PriorityBlockingQueue;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
import org.junit.Assert;
import org.junit.Test;
@@ -33,24 +33,12 @@ import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.AsyncDelegateRequestHandler;
import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.LocalContainerFactory;
import org.apache.tez.dag.app.rm.LocalTaskSchedulerService.TaskRequest;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
public class TestLocalTaskScheduler {
- public AppContext createMockAppContext() {
-
- ApplicationId appId = ApplicationId.newInstance(2000, 1);
- ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
-
- AppContext appContext = mock(AppContext.class);
- doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
-
- return appContext;
- }
@Test(timeout = 5000)
public void maxTasksAllocationsCannotBeExceeded() {
@@ -59,17 +47,24 @@ public class TestLocalTaskScheduler {
TezConfiguration tezConf = new TezConfiguration();
tezConf.setInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS, MAX_TASKS);
- LocalContainerFactory containerFactory = new LocalContainerFactory(createMockAppContext(), 1000);
+ ApplicationId appId = ApplicationId.newInstance(2000, 1);
+ ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+
+ TaskSchedulerContext
+ mockContext = TestTaskSchedulerHelpers.setupMockTaskSchedulerContext("", 0, "", true,
+ appAttemptId, 1000l, null, new Configuration());
+
+ LocalContainerFactory containerFactory = new LocalContainerFactory(appAttemptId, 1000);
+
HashMap<Object, Container> taskAllocations = new LinkedHashMap<Object, Container>();
PriorityBlockingQueue<TaskRequest> taskRequestQueue = new PriorityBlockingQueue<TaskRequest>();
- TaskSchedulerAppCallback appClientDelegate = mock(TaskSchedulerAppCallback.class);
// Object under test
AsyncDelegateRequestHandler requestHandler =
new AsyncDelegateRequestHandler(taskRequestQueue,
containerFactory,
taskAllocations,
- appClientDelegate,
+ mockContext,
tezConf);
// Allocate up to max tasks
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
index b555c62..c637f5f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
@@ -27,11 +27,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.dag.Task;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
import org.apache.tez.dag.app.rm.TestLocalTaskSchedulerService.MockLocalTaskSchedulerSerivce.MockAsyncDelegateRequestHandler;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
import org.junit.Assert;
import org.junit.Test;
@@ -82,14 +80,15 @@ public class TestLocalTaskSchedulerService {
* Normal flow of TaskAttempt
*/
@Test(timeout = 5000)
- public void testDeallocationBeforeAllocation() {
- AppContext appContext = mock(AppContext.class);
+ public void testDeallocationBeforeAllocation() throws InterruptedException {
ApplicationAttemptId appAttemptId =
ApplicationAttemptId.newInstance(ApplicationId.newInstance(10000l, 1), 1);
- doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
- MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce
- (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", appContext);
- taskSchedulerService.init(new Configuration());
+
+ TaskSchedulerContext mockContext = TestTaskSchedulerHelpers
+ .setupMockTaskSchedulerContext("", 0, "", false, appAttemptId, 10000l, null, new Configuration());
+
+ MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce(mockContext);
+ taskSchedulerService.initialize();
taskSchedulerService.start();
Task task = mock(Task.class);
@@ -103,21 +102,24 @@ public class TestLocalTaskSchedulerService {
assertEquals(1, requestHandler.deallocateCount);
// The corresponding AllocateTaskRequest will be removed, so won't been processed.
assertEquals(0, requestHandler.allocateCount);
- taskSchedulerService.stop();
+ taskSchedulerService.shutdown();
}
/**
* TaskAttempt Killed from START_WAIT
*/
@Test(timeout = 5000)
- public void testDeallocationAfterAllocation() {
- AppContext appContext = mock(AppContext.class);
+ public void testDeallocationAfterAllocation() throws InterruptedException {
ApplicationAttemptId appAttemptId =
ApplicationAttemptId.newInstance(ApplicationId.newInstance(10000l, 1), 1);
- doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
- MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce
- (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", appContext);
- taskSchedulerService.init(new Configuration());
+
+ TaskSchedulerContext mockContext = TestTaskSchedulerHelpers
+ .setupMockTaskSchedulerContext("", 0, "", false, appAttemptId, 10000l, null, new Configuration());
+
+ MockLocalTaskSchedulerSerivce taskSchedulerService =
+ new MockLocalTaskSchedulerSerivce(mockContext);
+
+ taskSchedulerService.initialize();
taskSchedulerService.start();
Task task = mock(Task.class);
@@ -130,33 +132,29 @@ public class TestLocalTaskSchedulerService {
requestHandler.drainRequest(2);
assertEquals(1, requestHandler.deallocateCount);
assertEquals(1, requestHandler.allocateCount);
- taskSchedulerService.stop();
+ taskSchedulerService.shutdown();
}
static class MockLocalTaskSchedulerSerivce extends LocalTaskSchedulerService {
private MockAsyncDelegateRequestHandler requestHandler;
- public MockLocalTaskSchedulerSerivce(TaskSchedulerAppCallback appClient,
- ContainerSignatureMatcher containerSignatureMatcher,
- String appHostName, int appHostPort, String appTrackingUrl,
- AppContext appContext) {
- super(appClient, containerSignatureMatcher, appHostName, appHostPort,
- appTrackingUrl, 10000l, appContext);
+ public MockLocalTaskSchedulerSerivce(TaskSchedulerContext appClient) {
+ super(appClient);
}
@Override
public AsyncDelegateRequestHandler createRequestHandler(Configuration conf) {
requestHandler = new MockAsyncDelegateRequestHandler(taskRequestQueue,
- new LocalContainerFactory(appContext, customContainerAppId),
+ new LocalContainerFactory(getContext().getApplicationAttemptId(), customContainerAppId),
taskAllocations,
- appClientDelegate,
+ getContext(),
conf);
return requestHandler;
}
@Override
- public void serviceStart() {
+ public void start() {
// don't start RequestHandler thread, control it in unit test
}
@@ -178,7 +176,7 @@ public class TestLocalTaskSchedulerService {
BlockingQueue<TaskRequest> taskRequestQueue,
LocalContainerFactory localContainerFactory,
HashMap<Object, Container> taskAllocations,
- TaskSchedulerAppCallback appClientDelegate, Configuration conf) {
+ TaskSchedulerContext appClientDelegate, Configuration conf) {
super(taskRequestQueue, localContainerFactory, taskAllocations,
appClientDelegate, conf);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
index 807e772..123a4d7 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
@@ -18,6 +18,8 @@
package org.apache.tez.dag.app.rm;
+import static org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.createCountingExecutingService;
+import static org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.setupMockTaskSchedulerContext;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@@ -42,8 +44,11 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@@ -59,23 +64,21 @@ import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tez.common.ContainerSignatureMatcher;
import org.apache.tez.common.MockDNSToSwitchMapping;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.DAGAppMasterState;
import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.HeldContainer;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
-import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerAppCallbackDrainable;
-import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableAppCallback;
+import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerContextDrainable;
+import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableContext;
import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.AlwaysMatchesContainerMatcher;
import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.PreemptionMatcher;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus;
+import org.junit.After;
import org.junit.Assert;
+import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
@@ -88,23 +91,39 @@ import com.google.common.collect.Sets;
public class TestTaskScheduler {
- RecordFactory recordFactory =
- RecordFactoryProvider.getRecordFactory(null);
-
static ContainerSignatureMatcher containerSignatureMatcher = new AlwaysMatchesContainerMatcher();
+ private ExecutorService contextCallbackExecutor;
@BeforeClass
public static void beforeClass() {
MockDNSToSwitchMapping.initializeMockRackResolver();
}
+ @Before
+ public void preTest() {
+ contextCallbackExecutor = Executors.newSingleThreadExecutor(
+ new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d")
+ .setDaemon(true)
+ .build());
+ }
+
+ @After
+ public void postTest() {
+ contextCallbackExecutor.shutdownNow();
+ }
+
+ private TaskSchedulerContextDrainable createDrainableContext(
+ TaskSchedulerContext taskSchedulerContext) {
+ TaskSchedulerContextImplWrapper wrapper =
+ new TaskSchedulerContextImplWrapper(taskSchedulerContext,
+ createCountingExecutingService(contextCallbackExecutor));
+ return new TaskSchedulerContextDrainable(wrapper);
+ }
+
@SuppressWarnings({ "unchecked" })
@Test(timeout=10000)
public void testTaskSchedulerNoReuse() throws Exception {
RackResolver.init(new YarnConfiguration());
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
- AppContext mockAppContext = mock(AppContext.class);
- when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
mock(TezAMRMClientAsync.class);
@@ -112,18 +131,19 @@ public class TestTaskScheduler {
String appHost = "host";
int appPort = 0;
String appUrl = "url";
- TaskSchedulerWithDrainableAppCallback scheduler =
- new TaskSchedulerWithDrainableAppCallback(
- mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
- appUrl, mockRMClient, mockAppContext);
- TaskSchedulerAppCallbackDrainable drainableAppCallback = scheduler
- .getDrainableAppCallback();
Configuration conf = new Configuration();
conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false);
int interval = 100;
conf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, interval);
- scheduler.init(conf);
+
+ TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf);
+ TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp);
+
+ TaskSchedulerWithDrainableContext scheduler =
+ new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient);
+
+ scheduler.initialize();
drainableAppCallback.drain();
verify(mockRMClient).init(conf);
verify(mockRMClient).setHeartbeatInterval(interval);
@@ -495,22 +515,18 @@ public class TestTaskScheduler {
AppFinalStatus finalStatus =
new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
- scheduler.stop();
+ scheduler.shutdown();
drainableAppCallback.drain();
verify(mockRMClient).
unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
- appMsg, appUrl);
+ appMsg, appUrl);
verify(mockRMClient).stop();
- scheduler.close();
}
@SuppressWarnings({ "unchecked" })
@Test(timeout=10000)
public void testTaskSchedulerWithReuse() throws Exception {
RackResolver.init(new YarnConfiguration());
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
- AppContext mockAppContext = mock(AppContext.class);
- when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
mock(TezAMRMClientAsync.class);
@@ -518,12 +534,6 @@ public class TestTaskScheduler {
String appHost = "host";
int appPort = 0;
String appUrl = "url";
- TaskSchedulerWithDrainableAppCallback scheduler =
- new TaskSchedulerWithDrainableAppCallback(
- mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
- appUrl, mockRMClient, mockAppContext);
- final TaskSchedulerAppCallbackDrainable drainableAppCallback = scheduler
- .getDrainableAppCallback();
Configuration conf = new Configuration();
// to match all in the same pass
@@ -531,7 +541,15 @@ public class TestTaskScheduler {
// to release immediately after deallocate
conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, 0);
conf.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, 0);
- scheduler.init(conf);
+
+ TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf);
+ final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp);
+
+ TaskSchedulerWithDrainableContext scheduler =
+ new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient);
+
+
+ scheduler.initialize();
drainableAppCallback.drain();
RegisterApplicationMasterResponse mockRegResponse =
@@ -992,23 +1010,18 @@ public class TestTaskScheduler {
AppFinalStatus finalStatus =
new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
- scheduler.stop();
+ scheduler.shutdown();
drainableAppCallback.drain();
verify(mockRMClient).
unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
- appMsg, appUrl);
+ appMsg, appUrl);
verify(mockRMClient).stop();
- scheduler.close();
}
@SuppressWarnings("unchecked")
@Test (timeout=5000)
public void testTaskSchedulerDetermineMinHeldContainers() throws Exception {
RackResolver.init(new YarnConfiguration());
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
- AppContext mockAppContext = mock(AppContext.class);
- when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
- when(mockAppContext.isSession()).thenReturn(true);
TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
mock(TezAMRMClientAsync.class);
@@ -1016,15 +1029,15 @@ public class TestTaskScheduler {
String appHost = "host";
int appPort = 0;
String appUrl = "url";
- TaskSchedulerWithDrainableAppCallback scheduler =
- new TaskSchedulerWithDrainableAppCallback(
- mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
- appUrl, mockRMClient, mockAppContext);
- TaskSchedulerAppCallbackDrainable drainableAppCallback = scheduler
- .getDrainableAppCallback();
- Configuration conf = new Configuration();
- scheduler.init(conf);
+ TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, true,
+ new Configuration());
+ final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp);
+
+ TaskSchedulerWithDrainableContext scheduler =
+ new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient);
+
+ scheduler.initialize();
RegisterApplicationMasterResponse mockRegResponse = mock(RegisterApplicationMasterResponse.class);
Resource mockMaxResource = mock(Resource.class);
Map<ApplicationAccessType, String> mockAcls = mock(Map.class);
@@ -1176,17 +1189,13 @@ public class TestTaskScheduler {
AppFinalStatus finalStatus =
new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
- scheduler.stop();
- scheduler.close();
+ scheduler.shutdown();
}
@SuppressWarnings("unchecked")
@Test(timeout=5000)
public void testTaskSchedulerRandomReuseExpireTime() throws Exception {
RackResolver.init(new YarnConfiguration());
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
- AppContext mockAppContext = mock(AppContext.class);
- when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
mock(TezAMRMClientAsync.class);
@@ -1194,25 +1203,31 @@ public class TestTaskScheduler {
String appHost = "host";
int appPort = 0;
String appUrl = "url";
- TaskSchedulerWithDrainableAppCallback scheduler1 =
- new TaskSchedulerWithDrainableAppCallback(
- mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
- appUrl, mockRMClient, mockAppContext);
- TaskSchedulerWithDrainableAppCallback scheduler2 =
- new TaskSchedulerWithDrainableAppCallback(
- mockApp, new AlwaysMatchesContainerMatcher(), appHost, appPort,
- appUrl, mockRMClient, mockAppContext);
long minTime = 1000l;
long maxTime = 100000l;
Configuration conf1 = new Configuration();
conf1.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, minTime);
conf1.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, minTime);
- scheduler1.init(conf1);
+
Configuration conf2 = new Configuration();
conf2.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MIN_MILLIS, minTime);
conf2.setLong(TezConfiguration.TEZ_AM_CONTAINER_IDLE_RELEASE_TIMEOUT_MAX_MILLIS, maxTime);
- scheduler2.init(conf2);
+
+ TaskSchedulerContext mockApp1 = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf1);
+ TaskSchedulerContext mockApp2 = setupMockTaskSchedulerContext(appHost, appPort, appUrl, conf2);
+ final TaskSchedulerContextDrainable drainableAppCallback1 = createDrainableContext(mockApp1);
+ final TaskSchedulerContextDrainable drainableAppCallback2 = createDrainableContext(mockApp2);
+
+
+ TaskSchedulerWithDrainableContext scheduler1 =
+ new TaskSchedulerWithDrainableContext(drainableAppCallback1, mockRMClient);
+ TaskSchedulerWithDrainableContext scheduler2 =
+ new TaskSchedulerWithDrainableContext(drainableAppCallback2, mockRMClient);
+
+ scheduler1.initialize();
+ scheduler2.initialize();
+
RegisterApplicationMasterResponse mockRegResponse =
mock(RegisterApplicationMasterResponse.class);
@@ -1250,20 +1265,16 @@ public class TestTaskScheduler {
String appMsg = "success";
AppFinalStatus finalStatus =
new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, appMsg, appUrl);
- when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
- scheduler1.stop();
- scheduler1.close();
- scheduler2.stop();
- scheduler2.close();
+ when(mockApp1.getFinalAppStatus()).thenReturn(finalStatus);
+ when(mockApp2.getFinalAppStatus()).thenReturn(finalStatus);
+ scheduler1.shutdown();
+ scheduler2.shutdown();
}
@SuppressWarnings({ "unchecked", "rawtypes" })
@Test (timeout=5000)
public void testTaskSchedulerPreemption() throws Exception {
RackResolver.init(new YarnConfiguration());
- TaskSchedulerAppCallback mockApp = mock(TaskSchedulerAppCallback.class);
- AppContext mockAppContext = mock(AppContext.class);
- when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
TezAMRMClientAsync<CookieContainerRequest> mockRMClient =
mock(TezAMRMClientAsync.class);
@@ -1271,16 +1282,18 @@ public class TestTaskScheduler {
String appHost = "host";
int appPort = 0;
String appUrl = "url";
- final TaskSchedulerWithDrainableAppCallback scheduler =
- new TaskSchedulerWithDrainableAppCallback(
- mockApp, new PreemptionMatcher(), appHost, appPort,
- appUrl, mockRMClient, mockAppContext);
- TaskSchedulerAppCallbackDrainable drainableAppCallback = scheduler
- .getDrainableAppCallback();
Configuration conf = new Configuration();
conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false);
- scheduler.init(conf);
+
+ TaskSchedulerContext mockApp = setupMockTaskSchedulerContext(appHost, appPort, appUrl, false,
+ null, null, new PreemptionMatcher(), conf);
+ final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(mockApp);
+
+ final TaskSchedulerWithDrainableContext scheduler =
+ new TaskSchedulerWithDrainableContext(drainableAppCallback, mockRMClient);
+
+ scheduler.initialize();
RegisterApplicationMasterResponse mockRegResponse =
mock(RegisterApplicationMasterResponse.class);
@@ -1530,7 +1543,7 @@ public class TestTaskScheduler {
scheduler.getProgress();
scheduler.getProgress();
scheduler.getProgress();
- verify(mockRMClient, times(2)).releaseAssignedContainer((ContainerId)any());
+ verify(mockRMClient, times(2)).releaseAssignedContainer((ContainerId) any());
scheduler.getProgress();
drainableAppCallback.drain();
// Next oldest mockTaskPri3KillA gets preempted to clear 10% of outstanding running preemptable tasks
@@ -1540,9 +1553,8 @@ public class TestTaskScheduler {
AppFinalStatus finalStatus =
new AppFinalStatus(FinalApplicationStatus.SUCCEEDED, "", appUrl);
when(mockApp.getFinalAppStatus()).thenReturn(finalStatus);
- scheduler.stop();
+ scheduler.shutdown();
drainableAppCallback.drain();
- scheduler.close();
}
@SuppressWarnings("unchecked")
@@ -1550,22 +1562,19 @@ public class TestTaskScheduler {
public void testLocalityMatching() throws Exception {
RackResolver.init(new Configuration());
- TaskSchedulerAppCallback appClient = mock(TaskSchedulerAppCallback.class);
TezAMRMClientAsync<CookieContainerRequest> amrmClient =
mock(TezAMRMClientAsync.class);
- AppContext mockAppContext = mock(AppContext.class);
- when(mockAppContext.getAMState()).thenReturn(DAGAppMasterState.RUNNING);
-
- TaskSchedulerWithDrainableAppCallback taskScheduler =
- new TaskSchedulerWithDrainableAppCallback(
- appClient, new AlwaysMatchesContainerMatcher(), "host", 0, "",
- amrmClient, mockAppContext);
- TaskSchedulerAppCallbackDrainable drainableAppCallback = taskScheduler
- .getDrainableAppCallback();
-
+
Configuration conf = new Configuration();
conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false);
- taskScheduler.init(conf);
+
+ TaskSchedulerContext appClient = setupMockTaskSchedulerContext("host", 0, "", conf);
+ final TaskSchedulerContextDrainable drainableAppCallback = createDrainableContext(appClient);
+
+ TaskSchedulerWithDrainableContext taskScheduler =
+ new TaskSchedulerWithDrainableContext(drainableAppCallback, amrmClient);
+
+ taskScheduler.initialize();
RegisterApplicationMasterResponse mockRegResponse = mock(RegisterApplicationMasterResponse.class);
Resource mockMaxResource = mock(Resource.class);
@@ -1693,7 +1702,7 @@ public class TestTaskScheduler {
AppFinalStatus finalStatus = new AppFinalStatus(
FinalApplicationStatus.SUCCEEDED, "", "");
when(appClient.getFinalAppStatus()).thenReturn(finalStatus);
- taskScheduler.close();
+ taskScheduler.shutdown();
}
@Test (timeout=5000)
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index 005692e..3ea0446 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -47,11 +47,13 @@ import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ContainerSignatureMatcher;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.client.DAGClientServer;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerContext;
+import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
import org.apache.tez.dag.app.dag.impl.TaskImpl;
import org.apache.tez.dag.app.dag.impl.VertexImpl;
@@ -61,10 +63,10 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
import org.apache.tez.dag.app.rm.container.AMContainerEventType;
import org.apache.tez.dag.app.rm.container.AMContainerMap;
import org.apache.tez.dag.app.rm.container.AMContainerState;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
import org.apache.tez.dag.app.web.WebUIService;
import org.apache.tez.dag.records.TaskAttemptTerminationCause;
import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -96,6 +98,7 @@ public class TestTaskSchedulerEventHandler {
protected void instantiateScheduelrs(String host, int port, String trackingUrl,
AppContext appContext) {
taskSchedulers[0] = mockTaskScheduler;
+ taskSchedulerServiceWrappers[0] = new ServicePluginLifecycleAbstractService(taskSchedulers[0]);
}
@Override
@@ -113,7 +116,7 @@ public class TestTaskSchedulerEventHandler {
TestEventHandler mockEventHandler;
ContainerSignatureMatcher mockSigMatcher;
MockTaskSchedulerEventHandler schedulerHandler;
- TaskSchedulerService mockTaskScheduler;
+ TaskScheduler mockTaskScheduler;
AMContainerMap mockAMContainerMap;
WebUIService mockWebUIService;
@@ -124,7 +127,7 @@ public class TestTaskSchedulerEventHandler {
mockClientService = mock(DAGClientServer.class);
mockEventHandler = new TestEventHandler();
mockSigMatcher = mock(ContainerSignatureMatcher.class);
- mockTaskScheduler = mock(TaskSchedulerService.class);
+ mockTaskScheduler = mock(TaskScheduler.class);
mockAMContainerMap = mock(AMContainerMap.class);
mockWebUIService = mock(WebUIService.class);
when(mockAppContext.getAllContainers()).thenReturn(mockAMContainerMap);
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index 04610ab..966c95a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -40,9 +40,13 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -54,13 +58,12 @@ import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ContainerSignatureMatcher;
import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
class TestTaskSchedulerHelpers {
@@ -134,12 +137,19 @@ class TestTaskSchedulerHelpers {
@Override
public void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
- taskSchedulers[0] = new TaskSchedulerWithDrainableAppCallback(new TaskSchedulerAppCallbackImpl(this, 0),
- containerSignatureMatcher, host, port, trackingUrl, amrmClientAsync,
- appContext);
- }
-
- public TaskSchedulerService getSpyTaskScheduler() {
+ TaskSchedulerContext taskSchedulerContext =
+ new TaskSchedulerContextImpl(this, appContext, 0, trackingUrl, 1000, host, port,
+ getConfig());
+ TaskSchedulerContextImplWrapper wrapper =
+ new TaskSchedulerContextImplWrapper(taskSchedulerContext,
+ new CountingExecutorService(appCallbackExecutor));
+ TaskSchedulerContextDrainable drainable = new TaskSchedulerContextDrainable(wrapper);
+ taskSchedulers[0] =
+ new TaskSchedulerWithDrainableContext(drainable, amrmClientAsync);
+ taskSchedulerServiceWrappers[0] = new ServicePluginLifecycleAbstractService(taskSchedulers[0]);
+ }
+
+ public TaskScheduler getSpyTaskScheduler() {
return taskSchedulers[0];
}
@@ -147,8 +157,8 @@ class TestTaskSchedulerHelpers {
public void serviceStart() {
instantiateScheduelrs("host", 0, "", appContext);
// Init the service so that reuse configuration is picked up.
- ((AbstractService)taskSchedulers[0]).init(getConfig());
- ((AbstractService)taskSchedulers[0]).start();
+ ((AbstractService)taskSchedulerServiceWrappers[0]).init(getConfig());
+ ((AbstractService)taskSchedulerServiceWrappers[0]).start();
taskSchedulers[0] = spy(taskSchedulers[0]);
}
@@ -188,61 +198,31 @@ class TestTaskSchedulerHelpers {
}
}
- static class TaskSchedulerWithDrainableAppCallback extends YarnTaskSchedulerService {
+ static class TaskSchedulerWithDrainableContext extends YarnTaskSchedulerService {
- private TaskSchedulerAppCallbackDrainable drainableAppCallback;
- public TaskSchedulerWithDrainableAppCallback(
- TaskSchedulerAppCallback appClient,
- ContainerSignatureMatcher containerSignatureMatcher,
- String appHostName, int appHostPort, String appTrackingUrl,
- AppContext appContext) {
- super(appClient, containerSignatureMatcher, appHostName, appHostPort,
- appTrackingUrl, appContext);
+ public TaskSchedulerWithDrainableContext(
+ TaskSchedulerContextDrainable appClient,
+ TezAMRMClientAsync<CookieContainerRequest> client) {
+ super(appClient, client);
shouldUnregister.set(true);
}
- public TaskSchedulerWithDrainableAppCallback(
- TaskSchedulerAppCallback appClient,
- ContainerSignatureMatcher containerSignatureMatcher,
- String appHostName, int appHostPort, String appTrackingUrl,
- TezAMRMClientAsync<CookieContainerRequest> client,
- AppContext appContext) {
- super(appClient, containerSignatureMatcher, appHostName, appHostPort,
- appTrackingUrl, client, appContext);
- shouldUnregister.set(true);
- }
-
- @Override
- TaskSchedulerAppCallback createAppCallbackDelegate(
- TaskSchedulerAppCallback realAppClient) {
- drainableAppCallback = new TaskSchedulerAppCallbackDrainable(
- new TaskSchedulerAppCallbackWrapper(realAppClient,
- appCallbackExecutor));
- return drainableAppCallback;
- }
-
- @Override
- ExecutorService createAppCallbackExecutorService() {
- ExecutorService real = super.createAppCallbackExecutorService();
- return new CountingExecutorService(real);
- }
-
- public TaskSchedulerAppCallbackDrainable getDrainableAppCallback() {
- return drainableAppCallback;
+ public TaskSchedulerContextDrainable getDrainableAppCallback() {
+ return (TaskSchedulerContextDrainable)getContext();
}
}
@SuppressWarnings("rawtypes")
- static class TaskSchedulerAppCallbackDrainable implements TaskSchedulerAppCallback {
+ static class TaskSchedulerContextDrainable implements TaskSchedulerContext {
int completedEvents;
int invocations;
- private TaskSchedulerAppCallback real;
+ private TaskSchedulerContext real;
private CountingExecutorService countingExecutorService;
final AtomicInteger count = new AtomicInteger(0);
- public TaskSchedulerAppCallbackDrainable(TaskSchedulerAppCallbackWrapper real) {
- countingExecutorService = (CountingExecutorService) real.executorService;
+ public TaskSchedulerContextDrainable(TaskSchedulerContextImplWrapper real) {
+ countingExecutorService = (CountingExecutorService) real.getExecutorService();
this.real = real;
}
@@ -303,6 +283,53 @@ class TestTaskSchedulerHelpers {
return real.getFinalAppStatus();
}
+ // Not incrementing invocations for methods which to not obtain locks,
+ // and do not go via the executor service.
+ @Override
+ public Configuration getInitialConfiguration() {
+ return real.getInitialConfiguration();
+ }
+
+ @Override
+ public String getAppTrackingUrl() {
+ return real.getAppTrackingUrl();
+ }
+
+ @Override
+ public long getCustomClusterIdentifier() {
+ return real.getCustomClusterIdentifier();
+ }
+
+ @Override
+ public ContainerSignatureMatcher getContainerSignatureMatcher() {
+ return real.getContainerSignatureMatcher();
+ }
+
+ @Override
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return real.getApplicationAttemptId();
+ }
+
+ @Override
+ public String getAppHostName() {
+ return real.getAppHostName();
+ }
+
+ @Override
+ public int getAppClientPort() {
+ return real.getAppClientPort();
+ }
+
+ @Override
+ public boolean isSession() {
+ return real.isSession();
+ }
+
+ @Override
+ public AMState getAMState() {
+ return real.getAMState();
+ }
+
@Override
public void preemptContainer(ContainerId cId) {
invocations++;
@@ -384,7 +411,11 @@ class TestTaskSchedulerHelpers {
}
}
}
-
+
+ static CountingExecutorService createCountingExecutingService(ExecutorService rawExecutor) {
+ return new CountingExecutorService(rawExecutor);
+ }
+
@SuppressWarnings({"rawtypes", "unchecked"})
private static class CountingExecutorService implements ExecutorService {
@@ -464,7 +495,50 @@ class TestTaskSchedulerHelpers {
throws InterruptedException, ExecutionException, TimeoutException {
throw new UnsupportedOperationException("Not expected to be used");
}
-
+ }
+
+ static TaskSchedulerContext setupMockTaskSchedulerContext(String appHost, int appPort,
+ String appUrl, Configuration conf) {
+ return setupMockTaskSchedulerContext(appHost, appPort, appUrl, false, conf);
+ }
+
+ static TaskSchedulerContext setupMockTaskSchedulerContext(String appHost, int appPort,
+ String appUrl, boolean isSession,
+ Configuration conf) {
+ return setupMockTaskSchedulerContext(appHost, appPort, appUrl, isSession, null, null, null,
+ conf);
+ }
+
+ static TaskSchedulerContext setupMockTaskSchedulerContext(String appHost, int appPort,
+ String appUrl, boolean isSession,
+ ApplicationAttemptId appAttemptId,
+ Long customAppIdentifier,
+ ContainerSignatureMatcher containerSignatureMatcher,
+ Configuration conf) {
+
+ TaskSchedulerContext mockContext = mock(TaskSchedulerContext.class);
+ when(mockContext.getAppHostName()).thenReturn(appHost);
+ when(mockContext.getAppClientPort()).thenReturn(appPort);
+ when(mockContext.getAppTrackingUrl()).thenReturn(appUrl);
+
+ when(mockContext.getAMState()).thenReturn(TaskSchedulerContext.AMState.RUNNING_APP);
+ when(mockContext.getInitialConfiguration()).thenReturn(conf);
+ when(mockContext.isSession()).thenReturn(isSession);
+ if (containerSignatureMatcher != null) {
+ when(mockContext.getContainerSignatureMatcher())
+ .thenReturn(containerSignatureMatcher);
+ } else {
+ when(mockContext.getContainerSignatureMatcher())
+ .thenReturn(new AlwaysMatchesContainerMatcher());
+ }
+ if (appAttemptId != null) {
+ when(mockContext.getApplicationAttemptId()).thenReturn(appAttemptId);
+ }
+ if (customAppIdentifier != null) {
+ when(mockContext.getCustomClusterIdentifier()).thenReturn(customAppIdentifier);
+ }
+
+ return mockContext;
}
}
[49/51] [abbrv] tez git commit: TEZ-2126. Add unit tests for
verifying multiple schedulers, launchers, communicators. (sseth)
Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
index b4064a0..352ad87 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java
@@ -28,17 +28,23 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Map;
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.dag.api.DagTypeConverters;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
import org.apache.tez.dag.api.VertexLocationHint;
import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerContext;
import org.apache.tez.dag.app.TaskAttemptListener;
@@ -47,6 +53,7 @@ import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.StateChangeNotifier;
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption;
+import org.apache.tez.runtime.api.ExecutionContext;
import org.junit.Test;
/**
@@ -60,7 +67,8 @@ public class TestVertexImpl2 {
Configuration conf = new TezConfiguration();
conf.set(TezConfiguration.TEZ_TASK_LOG_LEVEL, "DEBUG;org.apache.hadoop.ipc=INFO;org.apache.hadoop.server=INFO");
- LogTestInfoHolder testInfo = new LogTestInfoHolder(conf);
+ LogTestInfoHolder testInfo = new LogTestInfoHolder();
+ VertexWrapper vertexWrapper = createVertexWrapperForLogTests(testInfo, conf);
List<String> expectedCommands = new LinkedList<String>();
expectedCommands.add("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator");
@@ -71,7 +79,8 @@ public class TestVertexImpl2 {
TezConstants.TEZ_CONTAINER_LOGGER_NAME);
for (int i = 0 ; i < testInfo.numTasks ; i++) {
- ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+ ContainerContext containerContext = vertexWrapper
+ .vertex.getContainerContext(i);
String javaOpts = containerContext.getJavaOpts();
assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
for (String expectedCmd : expectedCommands) {
@@ -92,7 +101,8 @@ public class TestVertexImpl2 {
Configuration conf = new TezConfiguration();
conf.set(TezConfiguration.TEZ_TASK_LOG_LEVEL, "DEBUG");
- LogTestInfoHolder testInfo = new LogTestInfoHolder(conf);
+ LogTestInfoHolder testInfo = new LogTestInfoHolder();
+ VertexWrapper vertexWrapper = createVertexWrapperForLogTests(testInfo, conf);
List<String> expectedCommands = new LinkedList<String>();
expectedCommands.add("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator");
@@ -103,7 +113,7 @@ public class TestVertexImpl2 {
TezConstants.TEZ_CONTAINER_LOGGER_NAME);
for (int i = 0 ; i < testInfo.numTasks ; i++) {
- ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+ ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
String javaOpts = containerContext.getJavaOpts();
assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
for (String expectedCmd : expectedCommands) {
@@ -130,7 +140,8 @@ public class TestVertexImpl2 {
conf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LOG_LEVEL, "DEBUG;org.apache.tez=INFO");
conf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LAUNCH_CMD_OPTS, customJavaOpts);
- LogTestInfoHolder testInfo = new LogTestInfoHolder(conf);
+ LogTestInfoHolder testInfo = new LogTestInfoHolder();
+ VertexWrapper vertexWrapper = createVertexWrapperForLogTests(testInfo, conf);
// Expected command opts for regular tasks
List<String> expectedCommands = new LinkedList<String>();
@@ -142,7 +153,7 @@ public class TestVertexImpl2 {
TezConstants.TEZ_CONTAINER_LOGGER_NAME);
for (int i = 3 ; i < testInfo.numTasks ; i++) {
- ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+ ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
String javaOpts = containerContext.getJavaOpts();
assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
@@ -167,7 +178,7 @@ public class TestVertexImpl2 {
TezConstants.TEZ_CONTAINER_LOGGER_NAME);
for (int i = 0 ; i < 3 ; i++) {
- ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+ ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
String javaOpts = containerContext.getJavaOpts();
assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
@@ -195,7 +206,8 @@ public class TestVertexImpl2 {
conf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LOG_LEVEL, "DEBUG");
conf.set(TezConfiguration.TEZ_TASK_SPECIFIC_LAUNCH_CMD_OPTS, customJavaOpts);
- LogTestInfoHolder testInfo = new LogTestInfoHolder(conf);
+ LogTestInfoHolder testInfo = new LogTestInfoHolder();
+ VertexWrapper vertexWrapper = createVertexWrapperForLogTests(testInfo, conf);
// Expected command opts for regular tasks
List<String> expectedCommands = new LinkedList<String>();
@@ -207,7 +219,7 @@ public class TestVertexImpl2 {
TezConstants.TEZ_CONTAINER_LOGGER_NAME);
for (int i = 3 ; i < testInfo.numTasks ; i++) {
- ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+ ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
String javaOpts = containerContext.getJavaOpts();
assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
@@ -232,7 +244,7 @@ public class TestVertexImpl2 {
TezConstants.TEZ_CONTAINER_LOGGER_NAME);
for (int i = 0 ; i < 3 ; i++) {
- ContainerContext containerContext = testInfo.vertex.getContainerContext(i);
+ ContainerContext containerContext = vertexWrapper.vertex.getContainerContext(i);
String javaOpts = containerContext.getJavaOpts();
assertTrue(javaOpts.contains(testInfo.initialJavaOpts));
@@ -248,43 +260,224 @@ public class TestVertexImpl2 {
}
}
+ @Test(timeout = 5000)
+ public void testNullExecutionContexts() {
- private static class LogTestInfoHolder {
+ ExecutionContextTestInfoHolder info = new ExecutionContextTestInfoHolder(null, null);
+ VertexWrapper vertexWrapper = createVertexWrapperForExecutionContextTest(info);
- final AppContext mockAppContext;
- final DAG mockDag;
- final VertexImpl vertex;
- final DAGProtos.VertexPlan vertexPlan;
+ assertEquals(0, vertexWrapper.vertex.taskSchedulerIdentifier);
+ assertEquals(0, vertexWrapper.vertex.containerLauncherIdentifier);
+ assertEquals(0, vertexWrapper.vertex.taskCommunicatorIdentifier);
+ }
+
+ @Test(timeout = 5000)
+ public void testDefaultExecContextViaDag() {
+ VertexExecutionContext defaultExecContext = VertexExecutionContext.create(
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.TASK_SCHEDULER_NAME_BASE, 0),
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.CONTAINER_LAUNCHER_NAME_BASE, 2),
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.TASK_COMM_NAME_BASE, 2));
+ ExecutionContextTestInfoHolder info =
+ new ExecutionContextTestInfoHolder(null, defaultExecContext, 3);
+ VertexWrapper vertexWrapper = createVertexWrapperForExecutionContextTest(info);
+
+ assertEquals(0, vertexWrapper.vertex.taskSchedulerIdentifier);
+ assertEquals(2, vertexWrapper.vertex.containerLauncherIdentifier);
+ assertEquals(2, vertexWrapper.vertex.taskCommunicatorIdentifier);
+ }
+
+ @Test(timeout = 5000)
+ public void testVertexExecutionContextOnly() {
+ VertexExecutionContext vertexExecutionContext = VertexExecutionContext.create(
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.TASK_SCHEDULER_NAME_BASE, 1),
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.CONTAINER_LAUNCHER_NAME_BASE, 1),
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.TASK_COMM_NAME_BASE, 1));
+ ExecutionContextTestInfoHolder info =
+ new ExecutionContextTestInfoHolder(vertexExecutionContext, null, 3);
+ VertexWrapper vertexWrapper = createVertexWrapperForExecutionContextTest(info);
+
+ assertEquals(1, vertexWrapper.vertex.taskSchedulerIdentifier);
+ assertEquals(1, vertexWrapper.vertex.containerLauncherIdentifier);
+ assertEquals(1, vertexWrapper.vertex.taskCommunicatorIdentifier);
+ }
+
+ @Test(timeout = 5000)
+ public void testVertexExecutionContextOverride() {
+ VertexExecutionContext defaultExecContext = VertexExecutionContext.create(
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.TASK_SCHEDULER_NAME_BASE, 0),
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.CONTAINER_LAUNCHER_NAME_BASE, 2),
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.TASK_COMM_NAME_BASE, 2));
+
+ VertexExecutionContext vertexExecutionContext = VertexExecutionContext.create(
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.TASK_SCHEDULER_NAME_BASE, 1),
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.CONTAINER_LAUNCHER_NAME_BASE, 1),
+ ExecutionContextTestInfoHolder
+ .append(ExecutionContextTestInfoHolder.TASK_COMM_NAME_BASE, 1));
+ ExecutionContextTestInfoHolder info =
+ new ExecutionContextTestInfoHolder(vertexExecutionContext, defaultExecContext, 3);
+ VertexWrapper vertexWrapper = createVertexWrapperForExecutionContextTest(info);
+
+ assertEquals(1, vertexWrapper.vertex.taskSchedulerIdentifier);
+ assertEquals(1, vertexWrapper.vertex.containerLauncherIdentifier);
+ assertEquals(1, vertexWrapper.vertex.taskCommunicatorIdentifier);
+ }
+
+
+ private static class ExecutionContextTestInfoHolder {
+
+ static final String TASK_SCHEDULER_NAME_BASE = "TASK_SCHEDULER";
+ static final String CONTAINER_LAUNCHER_NAME_BASE = "CONTAINER_LAUNCHER";
+ static final String TASK_COMM_NAME_BASE = "TASK_COMMUNICATOR";
+
+ static String append(String base, int index) {
+ return base + index;
+ }
+
+ final String vertexName;
+ final VertexExecutionContext defaultExecutionContext;
+ final VertexExecutionContext vertexExecutionContext;
+ final BiMap<String, Integer> taskSchedulers = HashBiMap.create();
+ final BiMap<String, Integer> containerLaunchers = HashBiMap.create();
+ final BiMap<String, Integer> taskComms = HashBiMap.create();
+ final AppContext appContext;
+
+ public ExecutionContextTestInfoHolder(VertexExecutionContext vertexExecutionContext,
+ VertexExecutionContext defaultDagExecutionContext) {
+ this(vertexExecutionContext, defaultDagExecutionContext, 0);
+ }
+
+ public ExecutionContextTestInfoHolder(VertexExecutionContext vertexExecutionContext,
+ VertexExecutionContext defaultDagExecitionContext,
+ int numPlugins) {
+ this.vertexName = "testvertex";
+ this.vertexExecutionContext = vertexExecutionContext;
+ this.defaultExecutionContext = defaultDagExecitionContext;
+ if (numPlugins == 0) {
+ this.taskSchedulers.put(TezConstants.getTezYarnServicePluginName(), 0);
+ this.containerLaunchers.put(TezConstants.getTezYarnServicePluginName(), 0);
+ this.taskSchedulers.put(TezConstants.getTezYarnServicePluginName(), 0);
+ } else {
+ for (int i = 0; i < numPlugins; i++) {
+ this.taskSchedulers.put(append(TASK_SCHEDULER_NAME_BASE, i), i);
+ this.containerLaunchers.put(append(CONTAINER_LAUNCHER_NAME_BASE, i), i);
+ this.taskComms.put(append(TASK_COMM_NAME_BASE, i), i);
+ }
+ }
+
+ this.appContext = createDefaultMockAppContext();
+ DAG dag = appContext.getCurrentDAG();
+ doReturn(defaultDagExecitionContext).when(dag).getDefaultExecutionContext();
+ for (Map.Entry<String, Integer> entry : taskSchedulers.entrySet()) {
+ doReturn(entry.getKey()).when(appContext).getTaskSchedulerName(entry.getValue());
+ doReturn(entry.getValue()).when(appContext).getTaskScheduerIdentifier(entry.getKey());
+ }
+ for (Map.Entry<String, Integer> entry : containerLaunchers.entrySet()) {
+ doReturn(entry.getKey()).when(appContext).getContainerLauncherName(entry.getValue());
+ doReturn(entry.getValue()).when(appContext).getContainerLauncherIdentifier(entry.getKey());
+ }
+ for (Map.Entry<String, Integer> entry : taskComms.entrySet()) {
+ doReturn(entry.getKey()).when(appContext).getTaskCommunicatorName(entry.getValue());
+ doReturn(entry.getValue()).when(appContext).getTaskCommunicatorIdentifier(entry.getKey());
+ }
+ }
+ }
+ private VertexWrapper createVertexWrapperForExecutionContextTest(
+ ExecutionContextTestInfoHolder vertexInfo) {
+ VertexPlan vertexPlan = createVertexPlanForExeuctionContextTests(vertexInfo);
+ VertexWrapper vertexWrapper =
+ new VertexWrapper(vertexInfo.appContext, vertexPlan, new Configuration(false));
+ return vertexWrapper;
+ }
+
+ private VertexPlan createVertexPlanForExeuctionContextTests(ExecutionContextTestInfoHolder info) {
+ VertexPlan.Builder vertexPlanBuilder = VertexPlan.newBuilder()
+ .setName(info.vertexName)
+ .setTaskConfig(DAGProtos.PlanTaskConfiguration.newBuilder()
+ .setNumTasks(10)
+ .setJavaOpts("dontcare")
+ .setMemoryMb(1024)
+ .setVirtualCores(1)
+ .setTaskModule("taskmodule")
+ .build())
+ .setType(DAGProtos.PlanVertexType.NORMAL);
+ if (info.vertexExecutionContext != null) {
+ vertexPlanBuilder
+ .setExecutionContext(DagTypeConverters.convertToProto(info.vertexExecutionContext));
+ }
+ return vertexPlanBuilder.build();
+ }
+
+ private static class LogTestInfoHolder {
final int numTasks = 10;
final String initialJavaOpts = "initialJavaOpts";
final String envKey = "key1";
final String envVal = "val1";
+ final String vertexName;
+
+ public LogTestInfoHolder() {
+ this("testvertex");
+ }
- LogTestInfoHolder(Configuration conf) {
- this(conf, "testvertex");
+ public LogTestInfoHolder(String vertexName) {
+ this.vertexName = vertexName;
}
+ }
+
+ private VertexWrapper createVertexWrapperForLogTests(LogTestInfoHolder logTestInfoHolder,
+ Configuration conf) {
+ VertexPlan vertexPlan = createVertexPlanForLogTests(logTestInfoHolder);
+ VertexWrapper vertexWrapper = new VertexWrapper(vertexPlan, conf);
+ return vertexWrapper;
+ }
+
+ private VertexPlan createVertexPlanForLogTests(LogTestInfoHolder logTestInfoHolder) {
+ VertexPlan vertexPlan = VertexPlan.newBuilder()
+ .setName(logTestInfoHolder.vertexName)
+ .setTaskConfig(DAGProtos.PlanTaskConfiguration.newBuilder()
+ .setJavaOpts(logTestInfoHolder.initialJavaOpts)
+ .setNumTasks(logTestInfoHolder.numTasks)
+ .setMemoryMb(1024)
+ .setVirtualCores(1)
+ .setTaskModule("taskmodule")
+ .addEnvironmentSetting(DAGProtos.PlanKeyValuePair.newBuilder()
+ .setKey(logTestInfoHolder.envKey)
+ .setValue(logTestInfoHolder.envVal)
+ .build())
+ .build())
+ .setType(DAGProtos.PlanVertexType.NORMAL).build();
+ return vertexPlan;
+ }
+
+ private static class VertexWrapper {
- LogTestInfoHolder(Configuration conf, String vertexName) {
- mockAppContext = mock(AppContext.class);
- mockDag = mock(DAG.class);
- doReturn(new Credentials()).when(mockDag).getCredentials();
- doReturn(mockDag).when(mockAppContext).getCurrentDAG();
-
- vertexPlan = DAGProtos.VertexPlan.newBuilder()
- .setName(vertexName)
- .setTaskConfig(DAGProtos.PlanTaskConfiguration.newBuilder()
- .setJavaOpts(initialJavaOpts)
- .setNumTasks(numTasks)
- .setMemoryMb(1024)
- .setVirtualCores(1)
- .setTaskModule("taskmodule")
- .addEnvironmentSetting(DAGProtos.PlanKeyValuePair.newBuilder()
- .setKey(envKey)
- .setValue(envVal)
- .build())
- .build())
- .setType(DAGProtos.PlanVertexType.NORMAL).build();
+ final AppContext mockAppContext;
+ final VertexImpl vertex;
+ final VertexPlan vertexPlan;
+
+ VertexWrapper(AppContext appContext, VertexPlan vertexPlan, Configuration conf) {
+ if (appContext == null) {
+ mockAppContext = createDefaultMockAppContext();
+ DAG mockDag = mock(DAG.class);
+ doReturn(new Credentials()).when(mockDag).getCredentials();
+ doReturn(mockDag).when(mockAppContext).getCurrentDAG();
+ } else {
+ mockAppContext = appContext;
+ }
+
+
+ this.vertexPlan = vertexPlan;
vertex =
new VertexImpl(TezVertexID.fromString("vertex_1418197758681_0001_1_00"), vertexPlan,
@@ -293,5 +486,17 @@ public class TestVertexImpl2 {
VertexLocationHint.create(new LinkedList<TaskLocationHint>()), null,
new TaskSpecificLaunchCmdOption(conf), mock(StateChangeNotifier.class));
}
+
+ VertexWrapper(VertexPlan vertexPlan, Configuration conf) {
+ this(null, vertexPlan, conf);
+ }
+ }
+
+ private static AppContext createDefaultMockAppContext() {
+ AppContext appContext = mock(AppContext.class);
+ DAG mockDag = mock(DAG.class);
+ doReturn(new Credentials()).when(mockDag).getCredentials();
+ doReturn(mockDag).when(appContext).getCurrentDAG();
+ return appContext;
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
new file mode 100644
index 0000000..62a5f19
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherRouter.java
@@ -0,0 +1,361 @@
+/*
+ * 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.tez.dag.app.launcher;
+
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+public class TestContainerLauncherRouter {
+
+ @Before
+ @After
+ public void reset() {
+ ContainerLaucherRouterForMultipleLauncherTest.reset();
+ }
+
+ @Test(timeout = 5000)
+ public void testNoLaunchersSpecified() throws IOException {
+
+ AppContext appContext = mock(AppContext.class);
+ TaskAttemptListener tal = mock(TaskAttemptListener.class);
+
+ try {
+
+ new ContainerLaucherRouterForMultipleLauncherTest(appContext, tal, null, null,
+ false);
+ fail("Expecting a failure without any launchers being specified");
+ } catch (IllegalArgumentException e) {
+
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testCustomLauncherSpecified() throws IOException {
+ Configuration conf = new Configuration(false);
+
+ AppContext appContext = mock(AppContext.class);
+ TaskAttemptListener tal = mock(TaskAttemptListener.class);
+
+ String customLauncherName = "customLauncher";
+ List<NamedEntityDescriptor> launcherDescriptors = new LinkedList<>();
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ UserPayload customPayload = UserPayload.create(bb);
+ launcherDescriptors.add(
+ new NamedEntityDescriptor(customLauncherName, FakeContainerLauncher.class.getName())
+ .setUserPayload(customPayload));
+
+ ContainerLaucherRouterForMultipleLauncherTest clr =
+ new ContainerLaucherRouterForMultipleLauncherTest(appContext, tal, null,
+ launcherDescriptors,
+ true);
+ try {
+ clr.init(conf);
+ clr.start();
+
+ assertEquals(1, clr.getNumContainerLaunchers());
+ assertFalse(clr.getYarnContainerLauncherCreated());
+ assertFalse(clr.getUberContainerLauncherCreated());
+ assertEquals(customLauncherName, clr.getContainerLauncherName(0));
+ assertEquals(bb, clr.getContainerLauncherContext(0).getInitialUserPayload().getPayload());
+ } finally {
+ clr.stop();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testMultipleContainerLaunchers() throws IOException {
+ Configuration conf = new Configuration(false);
+ conf.set("testkey", "testvalue");
+ UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
+
+ AppContext appContext = mock(AppContext.class);
+ TaskAttemptListener tal = mock(TaskAttemptListener.class);
+
+ String customLauncherName = "customLauncher";
+ List<NamedEntityDescriptor> launcherDescriptors = new LinkedList<>();
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ UserPayload customPayload = UserPayload.create(bb);
+ launcherDescriptors.add(
+ new NamedEntityDescriptor(customLauncherName, FakeContainerLauncher.class.getName())
+ .setUserPayload(customPayload));
+ launcherDescriptors
+ .add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(userPayload));
+
+ ContainerLaucherRouterForMultipleLauncherTest clr =
+ new ContainerLaucherRouterForMultipleLauncherTest(appContext, tal, null,
+ launcherDescriptors,
+ true);
+ try {
+ clr.init(conf);
+ clr.start();
+
+ assertEquals(2, clr.getNumContainerLaunchers());
+ assertTrue(clr.getYarnContainerLauncherCreated());
+ assertFalse(clr.getUberContainerLauncherCreated());
+ assertEquals(customLauncherName, clr.getContainerLauncherName(0));
+ assertEquals(bb, clr.getContainerLauncherContext(0).getInitialUserPayload().getPayload());
+
+ assertEquals(TezConstants.getTezYarnServicePluginName(), clr.getContainerLauncherName(1));
+ Configuration confParsed = TezUtils
+ .createConfFromUserPayload(clr.getContainerLauncherContext(1).getInitialUserPayload());
+ assertEquals("testvalue", confParsed.get("testkey"));
+ } finally {
+ clr.stop();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testEventRouting() throws Exception {
+ Configuration conf = new Configuration(false);
+ UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
+
+ AppContext appContext = mock(AppContext.class);
+ TaskAttemptListener tal = mock(TaskAttemptListener.class);
+
+ String customLauncherName = "customLauncher";
+ List<NamedEntityDescriptor> launcherDescriptors = new LinkedList<>();
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ UserPayload customPayload = UserPayload.create(bb);
+ launcherDescriptors.add(
+ new NamedEntityDescriptor(customLauncherName, FakeContainerLauncher.class.getName())
+ .setUserPayload(customPayload));
+ launcherDescriptors
+ .add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(userPayload));
+
+ ContainerLaucherRouterForMultipleLauncherTest clr =
+ new ContainerLaucherRouterForMultipleLauncherTest(appContext, tal, null,
+ launcherDescriptors,
+ true);
+ try {
+ clr.init(conf);
+ clr.start();
+
+ assertEquals(2, clr.getNumContainerLaunchers());
+ assertTrue(clr.getYarnContainerLauncherCreated());
+ assertFalse(clr.getUberContainerLauncherCreated());
+ assertEquals(customLauncherName, clr.getContainerLauncherName(0));
+ assertEquals(TezConstants.getTezYarnServicePluginName(), clr.getContainerLauncherName(1));
+
+ verify(clr.getTestContainerLauncher(0)).initialize();
+ verify(clr.getTestContainerLauncher(0)).start();
+ verify(clr.getTestContainerLauncher(1)).initialize();
+ verify(clr.getTestContainerLauncher(1)).start();
+
+ ContainerLaunchContext clc1 = mock(ContainerLaunchContext.class);
+ Container container1 = mock(Container.class);
+
+ ContainerLaunchContext clc2 = mock(ContainerLaunchContext.class);
+ Container container2 = mock(Container.class);
+
+ NMCommunicatorLaunchRequestEvent launchRequestEvent1 =
+ new NMCommunicatorLaunchRequestEvent(clc1, container1, 0, 0, 0);
+ NMCommunicatorLaunchRequestEvent launchRequestEvent2 =
+ new NMCommunicatorLaunchRequestEvent(clc2, container2, 1, 0, 0);
+
+ clr.handle(launchRequestEvent1);
+
+
+ ArgumentCaptor<ContainerLaunchRequest> captor =
+ ArgumentCaptor.forClass(ContainerLaunchRequest.class);
+ verify(clr.getTestContainerLauncher(0)).launchContainer(captor.capture());
+ assertEquals(1, captor.getAllValues().size());
+ ContainerLaunchRequest launchRequest1 = captor.getValue();
+ assertEquals(clc1, launchRequest1.getContainerLaunchContext());
+
+ clr.handle(launchRequestEvent2);
+ captor = ArgumentCaptor.forClass(ContainerLaunchRequest.class);
+ verify(clr.getTestContainerLauncher(1)).launchContainer(captor.capture());
+ assertEquals(1, captor.getAllValues().size());
+ ContainerLaunchRequest launchRequest2 = captor.getValue();
+ assertEquals(clc2, launchRequest2.getContainerLaunchContext());
+
+ } finally {
+ clr.stop();
+ verify(clr.getTestContainerLauncher(0)).shutdown();
+ verify(clr.getTestContainerLauncher(1)).shutdown();
+ }
+ }
+
+ private static class ContainerLaucherRouterForMultipleLauncherTest
+ extends ContainerLauncherRouter {
+
+ // All variables setup as static since methods being overridden are invoked by the ContainerLauncherRouter ctor,
+ // and regular variables will not be initialized at this point.
+ private static final AtomicInteger numContainerLaunchers = new AtomicInteger(0);
+ private static final Set<Integer> containerLauncherIndices = new HashSet<>();
+ private static final ContainerLauncher yarnContainerLauncher = mock(ContainerLauncher.class);
+ private static final ContainerLauncher uberContainerlauncher = mock(ContainerLauncher.class);
+ private static final AtomicBoolean yarnContainerLauncherCreated = new AtomicBoolean(false);
+ private static final AtomicBoolean uberContainerLauncherCreated = new AtomicBoolean(false);
+
+ private static final List<ContainerLauncherContext> containerLauncherContexts =
+ new LinkedList<>();
+ private static final List<String> containerLauncherNames = new LinkedList<>();
+ private static final List<ContainerLauncher> testContainerLaunchers = new LinkedList<>();
+
+
+ public static void reset() {
+ numContainerLaunchers.set(0);
+ containerLauncherIndices.clear();
+ yarnContainerLauncherCreated.set(false);
+ uberContainerLauncherCreated.set(false);
+ containerLauncherContexts.clear();
+ containerLauncherNames.clear();
+ testContainerLaunchers.clear();
+ }
+
+ public ContainerLaucherRouterForMultipleLauncherTest(AppContext context,
+ TaskAttemptListener taskAttemptListener,
+ String workingDirectory,
+ List<NamedEntityDescriptor> containerLauncherDescriptors,
+ boolean isPureLocalMode) throws
+ UnknownHostException {
+ super(context, taskAttemptListener, workingDirectory,
+ containerLauncherDescriptors, isPureLocalMode);
+ }
+
+ @Override
+ ContainerLauncher createContainerLauncher(NamedEntityDescriptor containerLauncherDescriptor,
+ AppContext context,
+ ContainerLauncherContext containerLauncherContext,
+ TaskAttemptListener taskAttemptListener,
+ String workingDirectory,
+ int containerLauncherIndex,
+ boolean isPureLocalMode) throws
+ UnknownHostException {
+ numContainerLaunchers.incrementAndGet();
+ boolean added = containerLauncherIndices.add(containerLauncherIndex);
+ assertTrue("Cannot add multiple launchers with the same index", added);
+ containerLauncherNames.add(containerLauncherDescriptor.getEntityName());
+ containerLauncherContexts.add(containerLauncherContext);
+ return super
+ .createContainerLauncher(containerLauncherDescriptor, context, containerLauncherContext,
+ taskAttemptListener, workingDirectory, containerLauncherIndex, isPureLocalMode);
+ }
+
+ @Override
+ ContainerLauncher createYarnContainerLauncher(
+ ContainerLauncherContext containerLauncherContext) {
+ yarnContainerLauncherCreated.set(true);
+ testContainerLaunchers.add(yarnContainerLauncher);
+ return yarnContainerLauncher;
+ }
+
+ @Override
+ ContainerLauncher createUberContainerLauncher(ContainerLauncherContext containerLauncherContext,
+ AppContext context,
+ TaskAttemptListener taskAttemptListener,
+ String workingDirectory,
+ boolean isPureLocalMode) throws
+ UnknownHostException {
+ uberContainerLauncherCreated.set(true);
+ testContainerLaunchers.add(uberContainerlauncher);
+ return uberContainerlauncher;
+ }
+
+ @Override
+ ContainerLauncher createCustomContainerLauncher(
+ ContainerLauncherContext containerLauncherContext,
+ NamedEntityDescriptor containerLauncherDescriptor) {
+ ContainerLauncher spyLauncher = spy(super.createCustomContainerLauncher(
+ containerLauncherContext, containerLauncherDescriptor));
+ testContainerLaunchers.add(spyLauncher);
+ return spyLauncher;
+ }
+
+ public int getNumContainerLaunchers() {
+ return numContainerLaunchers.get();
+ }
+
+ public boolean getYarnContainerLauncherCreated() {
+ return yarnContainerLauncherCreated.get();
+ }
+
+ public boolean getUberContainerLauncherCreated() {
+ return uberContainerLauncherCreated.get();
+ }
+
+ public String getContainerLauncherName(int containerLauncherIndex) {
+ return containerLauncherNames.get(containerLauncherIndex);
+ }
+
+ public ContainerLauncher getTestContainerLauncher(int containerLauncherIndex) {
+ return testContainerLaunchers.get(containerLauncherIndex);
+ }
+
+ public ContainerLauncherContext getContainerLauncherContext(int containerLauncherIndex) {
+ return containerLauncherContexts.get(containerLauncherIndex);
+ }
+ }
+
+ private static class FakeContainerLauncher extends ContainerLauncher {
+
+ public FakeContainerLauncher(
+ ContainerLauncherContext containerLauncherContext) {
+ super(containerLauncherContext);
+ }
+
+ @Override
+ public void launchContainer(ContainerLaunchRequest launchRequest) {
+
+ }
+
+ @Override
+ public void stopContainer(ContainerStopRequest stopRequest) {
+
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index f8aa1e2..3e68a4c 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -19,22 +19,30 @@
package org.apache.tez.dag.app.rm;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
+import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
@@ -44,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.Event;
@@ -53,12 +62,13 @@ import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.client.DAGClientServer;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerContext;
import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
+import org.apache.tez.dag.app.dag.TaskAttempt;
import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
import org.apache.tez.dag.app.dag.impl.TaskImpl;
import org.apache.tez.dag.app.dag.impl.VertexImpl;
@@ -70,8 +80,14 @@ import org.apache.tez.dag.app.rm.container.AMContainerMap;
import org.apache.tez.dag.app.rm.container.AMContainerState;
import org.apache.tez.dag.app.web.WebUIService;
import org.apache.tez.dag.records.TaskAttemptTerminationCause;
+import org.apache.tez.dag.records.TezDAGID;
import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -95,10 +111,9 @@ public class TestTaskSchedulerEventHandler {
public MockTaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
- ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
- UserPayload defaultPayload) {
+ ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI,
- new LinkedList<NamedEntityDescriptor>(), defaultPayload, false);
+ Lists.newArrayList(new NamedEntityDescriptor("FakeDescriptor", null)), false);
}
@Override
@@ -140,14 +155,8 @@ public class TestTaskSchedulerEventHandler {
when(mockAppContext.getAllContainers()).thenReturn(mockAMContainerMap);
when(mockClientService.getBindAddress()).thenReturn(new InetSocketAddress(10000));
Configuration conf = new Configuration(false);
- UserPayload userPayload;
- try {
- userPayload = TezUtils.createUserPayloadFromConf(conf);
- } catch (IOException e) {
- throw new TezUncheckedException(e);
- }
schedulerHandler = new MockTaskSchedulerEventHandler(
- mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService, userPayload);
+ mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService);
}
@Test(timeout = 5000)
@@ -272,7 +281,7 @@ public class TestTaskSchedulerEventHandler {
when(mockAmContainer.getContainerLauncherIdentifier()).thenReturn(0);
when(mockAmContainer.getTaskCommunicatorIdentifier()).thenReturn(0);
ContainerId mockCId = mock(ContainerId.class);
- verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId)any());
+ verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId) any());
when(mockAMContainerMap.get(mockCId)).thenReturn(mockAmContainer);
schedulerHandler.preemptContainer(0, mockCId);
verify(mockTaskScheduler, times(1)).deallocateContainer(mockCId);
@@ -400,5 +409,300 @@ public class TestTaskSchedulerEventHandler {
}
- // TODO TEZ-2003. Add tests with multiple schedulers, and ensuring that events go out with correct IDs.
+ @Test(timeout = 5000)
+ public void testNoSchedulerSpecified() throws IOException {
+ try {
+ TSEHForMultipleSchedulersTest tseh =
+ new TSEHForMultipleSchedulersTest(mockAppContext, mockClientService, mockEventHandler,
+ mockSigMatcher, mockWebUIService, null, false);
+ fail("Expecting an IllegalStateException with no schedulers specified");
+ } catch (IllegalArgumentException e) {
+ }
+ }
+
+ // Verified via statics
+ @Test(timeout = 5000)
+ public void testCustomTaskSchedulerSetup() throws IOException {
+ Configuration conf = new Configuration(false);
+ conf.set("testkey", "testval");
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+ String customSchedulerName = "fakeScheduler";
+ List<NamedEntityDescriptor> taskSchedulers = new LinkedList<>();
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ UserPayload userPayload = UserPayload.create(bb);
+ taskSchedulers.add(
+ new NamedEntityDescriptor(customSchedulerName, FakeTaskScheduler.class.getName())
+ .setUserPayload(userPayload));
+ taskSchedulers.add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(defaultPayload));
+
+ TSEHForMultipleSchedulersTest tseh =
+ new TSEHForMultipleSchedulersTest(mockAppContext, mockClientService, mockEventHandler,
+ mockSigMatcher, mockWebUIService, taskSchedulers, false);
+
+ tseh.init(conf);
+ tseh.start();
+
+ // Verify that the YARN task scheduler is installed by default
+ assertTrue(tseh.getYarnSchedulerCreated());
+ assertFalse(tseh.getUberSchedulerCreated());
+ assertEquals(2, tseh.getNumCreateInvocations());
+
+ // Verify the order of the schedulers
+ assertEquals(customSchedulerName, tseh.getTaskSchedulerName(0));
+ assertEquals(TezConstants.getTezYarnServicePluginName(), tseh.getTaskSchedulerName(1));
+
+ // Verify the payload setup for the custom task scheduler
+ assertNotNull(tseh.getTaskSchedulerContext(0));
+ assertEquals(bb, tseh.getTaskSchedulerContext(0).getInitialUserPayload().getPayload());
+
+ // Verify the payload on the yarn scheduler
+ assertNotNull(tseh.getTaskSchedulerContext(1));
+ Configuration parsed = TezUtils.createConfFromUserPayload(tseh.getTaskSchedulerContext(1).getInitialUserPayload());
+ assertEquals("testval", parsed.get("testkey"));
+ }
+
+ @Test(timeout = 5000)
+ public void testTaskSchedulerRouting() throws Exception {
+ Configuration conf = new Configuration(false);
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+ String customSchedulerName = "fakeScheduler";
+ List<NamedEntityDescriptor> taskSchedulers = new LinkedList<>();
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ UserPayload userPayload = UserPayload.create(bb);
+ taskSchedulers.add(
+ new NamedEntityDescriptor(customSchedulerName, FakeTaskScheduler.class.getName())
+ .setUserPayload(userPayload));
+ taskSchedulers.add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(defaultPayload));
+
+ TSEHForMultipleSchedulersTest tseh =
+ new TSEHForMultipleSchedulersTest(mockAppContext, mockClientService, mockEventHandler,
+ mockSigMatcher, mockWebUIService, taskSchedulers, false);
+
+ tseh.init(conf);
+ tseh.start();
+
+ // Verify that the YARN task scheduler is installed by default
+ assertTrue(tseh.getYarnSchedulerCreated());
+ assertFalse(tseh.getUberSchedulerCreated());
+ assertEquals(2, tseh.getNumCreateInvocations());
+
+ // Verify the order of the schedulers
+ assertEquals(customSchedulerName, tseh.getTaskSchedulerName(0));
+ assertEquals(TezConstants.getTezYarnServicePluginName(), tseh.getTaskSchedulerName(1));
+
+ verify(tseh.getTestTaskScheduler(0)).initialize();
+ verify(tseh.getTestTaskScheduler(0)).start();
+
+ ApplicationId appId = ApplicationId.newInstance(1000, 1);
+ TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+ TezVertexID vertexID = TezVertexID.getInstance(dagId, 1);
+ TezTaskID taskId1 = TezTaskID.getInstance(vertexID, 1);
+ TezTaskAttemptID attemptId11 = TezTaskAttemptID.getInstance(taskId1, 1);
+ TezTaskID taskId2 = TezTaskID.getInstance(vertexID, 2);
+ TezTaskAttemptID attemptId21 = TezTaskAttemptID.getInstance(taskId2, 1);
+
+ Resource resource = Resource.newInstance(1024, 1);
+
+ TaskAttempt mockTaskAttempt1 = mock(TaskAttempt.class);
+ TaskAttempt mockTaskAttempt2 = mock(TaskAttempt.class);
+
+ AMSchedulerEventTALaunchRequest launchRequest1 =
+ new AMSchedulerEventTALaunchRequest(attemptId11, resource, mock(TaskSpec.class),
+ mockTaskAttempt1, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), 0, 0,
+ 0);
+
+ tseh.handle(launchRequest1);
+
+ verify(tseh.getTestTaskScheduler(0)).allocateTask(eq(mockTaskAttempt1), eq(resource),
+ any(String[].class), any(String[].class), any(Priority.class), any(Object.class),
+ eq(launchRequest1));
+
+ AMSchedulerEventTALaunchRequest launchRequest2 =
+ new AMSchedulerEventTALaunchRequest(attemptId21, resource, mock(TaskSpec.class),
+ mockTaskAttempt2, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), 1, 0,
+ 0);
+ tseh.handle(launchRequest2);
+ verify(tseh.getTestTaskScheduler(1)).allocateTask(eq(mockTaskAttempt2), eq(resource),
+ any(String[].class), any(String[].class), any(Priority.class), any(Object.class),
+ eq(launchRequest2));
+ }
+
+ private static class TSEHForMultipleSchedulersTest extends TaskSchedulerEventHandler {
+
+ private final TaskScheduler yarnTaskScheduler;
+ private final TaskScheduler uberTaskScheduler;
+ private final AtomicBoolean uberSchedulerCreated = new AtomicBoolean(false);
+ private final AtomicBoolean yarnSchedulerCreated = new AtomicBoolean(false);
+ private final AtomicInteger numCreateInvocations = new AtomicInteger(0);
+ private final Set<Integer> seenSchedulers = new HashSet<>();
+ private final List<TaskSchedulerContext> taskSchedulerContexts = new LinkedList<>();
+ private final List<String> taskSchedulerNames = new LinkedList<>();
+ private final List<TaskScheduler> testTaskSchedulers = new LinkedList<>();
+
+ public TSEHForMultipleSchedulersTest(AppContext appContext,
+ DAGClientServer clientService,
+ EventHandler eventHandler,
+ ContainerSignatureMatcher containerSignatureMatcher,
+ WebUIService webUI,
+ List<NamedEntityDescriptor> schedulerDescriptors,
+ boolean isPureLocalMode) {
+ super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI,
+ schedulerDescriptors, isPureLocalMode);
+ yarnTaskScheduler = mock(TaskScheduler.class);
+ uberTaskScheduler = mock(TaskScheduler.class);
+ }
+
+ @Override
+ TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
+ AppContext appContext,
+ NamedEntityDescriptor taskSchedulerDescriptor,
+ long customAppIdIdentifier,
+ int schedulerId) {
+
+ numCreateInvocations.incrementAndGet();
+ boolean added = seenSchedulers.add(schedulerId);
+ assertTrue("Cannot add multiple schedulers with the same schedulerId", added);
+ taskSchedulerNames.add(taskSchedulerDescriptor.getEntityName());
+ return super.createTaskScheduler(host, port, trackingUrl, appContext, taskSchedulerDescriptor,
+ customAppIdIdentifier, schedulerId);
+ }
+
+ @Override
+ TaskSchedulerContext wrapTaskSchedulerContext(TaskSchedulerContext rawContext) {
+ // Avoid wrapping in threads
+ return rawContext;
+ }
+
+ @Override
+ TaskScheduler createYarnTaskScheduler(TaskSchedulerContext taskSchedulerContext, int schedulerId) {
+ taskSchedulerContexts.add(taskSchedulerContext);
+ testTaskSchedulers.add(yarnTaskScheduler);
+ yarnSchedulerCreated.set(true);
+ return yarnTaskScheduler;
+ }
+
+ @Override
+ TaskScheduler createUberTaskScheduler(TaskSchedulerContext taskSchedulerContext, int schedulerId) {
+ taskSchedulerContexts.add(taskSchedulerContext);
+ uberSchedulerCreated.set(true);
+ testTaskSchedulers.add(yarnTaskScheduler);
+ return uberTaskScheduler;
+ }
+
+ @Override
+ TaskScheduler createCustomTaskScheduler(TaskSchedulerContext taskSchedulerContext,
+ NamedEntityDescriptor taskSchedulerDescriptor, int schedulerId) {
+ taskSchedulerContexts.add(taskSchedulerContext);
+ TaskScheduler taskScheduler = spy(super.createCustomTaskScheduler(taskSchedulerContext, taskSchedulerDescriptor, schedulerId));
+ testTaskSchedulers.add(taskScheduler);
+ return taskScheduler;
+ }
+
+ @Override
+ // Inline handling of events.
+ public void handle(AMSchedulerEvent event) {
+ handleEvent(event);
+ }
+
+ public boolean getUberSchedulerCreated() {
+ return uberSchedulerCreated.get();
+ }
+
+ public boolean getYarnSchedulerCreated() {
+ return yarnSchedulerCreated.get();
+ }
+
+ public int getNumCreateInvocations() {
+ return numCreateInvocations.get();
+ }
+
+ public TaskSchedulerContext getTaskSchedulerContext(int schedulerId) {
+ return taskSchedulerContexts.get(schedulerId);
+ }
+
+ public String getTaskSchedulerName(int schedulerId) {
+ return taskSchedulerNames.get(schedulerId);
+ }
+
+ public TaskScheduler getTestTaskScheduler(int schedulerId) {
+ return testTaskSchedulers.get(schedulerId);
+ }
+ }
+
+ public static class FakeTaskScheduler extends TaskScheduler {
+
+ public FakeTaskScheduler(
+ TaskSchedulerContext taskSchedulerContext) {
+ super(taskSchedulerContext);
+ }
+
+ @Override
+ public Resource getAvailableResources() {
+ return null;
+ }
+
+ @Override
+ public int getClusterNodeCount() {
+ return 0;
+ }
+
+ @Override
+ public void dagComplete() {
+
+ }
+
+ @Override
+ public Resource getTotalResources() {
+ return null;
+ }
+
+ @Override
+ public void blacklistNode(NodeId nodeId) {
+
+ }
+
+ @Override
+ public void unblacklistNode(NodeId nodeId) {
+
+ }
+
+ @Override
+ public void allocateTask(Object task, Resource capability, String[] hosts, String[] racks,
+ Priority priority, Object containerSignature, Object clientCookie) {
+
+ }
+
+ @Override
+ public void allocateTask(Object task, Resource capability, ContainerId containerId,
+ Priority priority, Object containerSignature, Object clientCookie) {
+
+ }
+
+ @Override
+ public boolean deallocateTask(Object task, boolean taskSucceeded,
+ TaskAttemptEndReason endReason) {
+ return false;
+ }
+
+ @Override
+ public Object deallocateContainer(ContainerId containerId) {
+ return null;
+ }
+
+ @Override
+ public void setShouldUnregister() {
+
+ }
+
+ @Override
+ public boolean hasUnregistered() {
+ return false;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index 59ab00a..0746507 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
@@ -138,7 +139,8 @@ class TestTaskSchedulerHelpers {
ContainerSignatureMatcher containerSignatureMatcher,
UserPayload defaultPayload) {
super(appContext, null, eventHandler, containerSignatureMatcher, null,
- new LinkedList<NamedEntityDescriptor>(), defaultPayload, false);
+ Lists.newArrayList(new NamedEntityDescriptor("FakeScheduler", null)),
+ false);
this.amrmClientAsync = amrmClientAsync;
this.containerSignatureMatcher = containerSignatureMatcher;
this.defaultPayload = defaultPayload;
[11/51] [abbrv] tez git commit: TEZ-2139. Update version to
0.7.0-TEZ-2003-SNAPSHOT. (sseth)
Posted by ss...@apache.org.
TEZ-2139. Update version to 0.7.0-TEZ-2003-SNAPSHOT. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/6db0fde0
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/6db0fde0
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/6db0fde0
Branch: refs/heads/TEZ-2003
Commit: 6db0fde0b9df95eca440c4c8d1fe2d20ddc0b6c0
Parents: 5150c20
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Feb 23 21:59:39 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
docs/pom.xml | 2 +-
pom.xml | 2 +-
tez-api/pom.xml | 2 +-
tez-common/pom.xml | 2 +-
tez-dag/pom.xml | 2 +-
tez-dist/pom.xml | 2 +-
tez-examples/pom.xml | 2 +-
tez-ext-service-tests/pom.xml | 2 +-
tez-mapreduce/pom.xml | 2 +-
tez-plugins/pom.xml | 2 +-
tez-plugins/tez-mbeans-resource-calculator/pom.xml | 2 +-
tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 2 +-
tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +-
tez-runtime-internals/pom.xml | 2 +-
tez-runtime-library/pom.xml | 2 +-
tez-tests/pom.xml | 2 +-
tez-ui/pom.xml | 2 +-
18 files changed, 18 insertions(+), 17 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 4377f57..5c5fd8e 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -8,5 +8,6 @@ ALL CHANGES:
TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
TEZ-2131. Add additional tests for tasks running in the AM.
TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
+ TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index f80b5c1..def3273 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -27,7 +27,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-docs</artifactId>
<packaging>pom</packaging>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2ddb113..1ced4e7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
<packaging>pom</packaging>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
<name>tez</name>
<licenses>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-api/pom.xml b/tez-api/pom.xml
index bb09c6d..93947ab 100644
--- a/tez-api/pom.xml
+++ b/tez-api/pom.xml
@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-api</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-common/pom.xml
----------------------------------------------------------------------
diff --git a/tez-common/pom.xml b/tez-common/pom.xml
index c6922be..5e0ceef 100644
--- a/tez-common/pom.xml
+++ b/tez-common/pom.xml
@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-common</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index 355ca74..00efdd6 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -19,7 +19,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<properties>
<tez.component>tez-dag</tez.component>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml
index e78cb8e..e188fe2 100644
--- a/tez-dist/pom.xml
+++ b/tez-dist/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-dist</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-examples/pom.xml
----------------------------------------------------------------------
diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml
index cd2df5c..d15e966 100644
--- a/tez-examples/pom.xml
+++ b/tez-examples/pom.xml
@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-examples</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index 37f68b1..1113341 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -20,7 +20,7 @@
<parent>
<artifactId>tez</artifactId>
<groupId>org.apache.tez</groupId>
- <version>0.7.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<!-- TODO TEZ-2003 Merge this into the tez-tests module -->
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index 300f781..af8bc8b 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-mapreduce</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml
index f5ec3b4..fe6f110 100644
--- a/tez-plugins/pom.xml
+++ b/tez-plugins/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-plugins</artifactId>
<packaging>pom</packaging>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-plugins/tez-mbeans-resource-calculator/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-mbeans-resource-calculator/pom.xml b/tez-plugins/tez-mbeans-resource-calculator/pom.xml
index 4be7f28..5c828a4 100644
--- a/tez-plugins/tez-mbeans-resource-calculator/pom.xml
+++ b/tez-plugins/tez-mbeans-resource-calculator/pom.xml
@@ -19,7 +19,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez-plugins</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-mbeans-resource-calculator</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
index aeae5cf..761bc10 100644
--- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
+++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml
@@ -19,7 +19,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez-plugins</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-yarn-timeline-history-with-acls</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-plugins/tez-yarn-timeline-history/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml
index ca76988..1e40329 100644
--- a/tez-plugins/tez-yarn-timeline-history/pom.xml
+++ b/tez-plugins/tez-yarn-timeline-history/pom.xml
@@ -19,7 +19,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez-plugins</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-yarn-timeline-history</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-runtime-internals/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml
index 3756fa9..b6dee8c 100644
--- a/tez-runtime-internals/pom.xml
+++ b/tez-runtime-internals/pom.xml
@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-runtime-internals</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-runtime-library/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
index 25f1cc1..c0924d4 100644
--- a/tez-runtime-library/pom.xml
+++ b/tez-runtime-library/pom.xml
@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-runtime-library</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml
index 91bc753..530ab77 100644
--- a/tez-tests/pom.xml
+++ b/tez-tests/pom.xml
@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-tests</artifactId>
http://git-wip-us.apache.org/repos/asf/tez/blob/6db0fde0/tez-ui/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml
index bfc6f01..8ecf7d0 100644
--- a/tez-ui/pom.xml
+++ b/tez-ui/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-ui</artifactId>
<packaging>war</packaging>
[03/51] [abbrv] tez git commit: TEZ-2090. Add tests for jobs running
in external services. (sseth)
Posted by ss...@apache.org.
TEZ-2090. Add tests for jobs running in external services. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/36e7f854
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/36e7f854
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/36e7f854
Branch: refs/heads/TEZ-2003
Commit: 36e7f854b8c648abf7aff1b5dca806f0d72108c9
Parents: 7ab75d8
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 13 17:24:05 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:22:32 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
pom.xml | 6 +
.../apache/tez/dag/api/TezConfiguration.java | 2 +
.../apache/tez/dag/api/TaskCommunicator.java | 1 +
.../tez/dag/api/TaskCommunicatorContext.java | 3 +
.../tez/dag/app/TezTaskCommunicatorImpl.java | 42 +-
.../dag/app/rm/TaskSchedulerEventHandler.java | 2 +-
tez-ext-service-tests/pom.xml | 161 ++++
.../tez/dag/app/TezTestServiceCommunicator.java | 152 ++++
.../TezTestServiceContainerLauncher.java | 144 ++++
.../TezTestServiceNoOpContainerLauncher.java | 66 ++
.../rm/TezTestServiceTaskSchedulerService.java | 347 ++++++++
.../TezTestServiceTaskCommunicatorImpl.java | 182 ++++
.../org/apache/tez/service/ContainerRunner.java | 27 +
.../tez/service/MiniTezTestServiceCluster.java | 163 ++++
.../service/TezTestServiceConfConstants.java | 41 +
.../TezTestServiceProtocolBlockingPB.java | 22 +
.../tez/service/impl/ContainerRunnerImpl.java | 512 +++++++++++
.../apache/tez/service/impl/TezTestService.java | 126 +++
.../impl/TezTestServiceProtocolClientImpl.java | 82 ++
.../impl/TezTestServiceProtocolServerImpl.java | 133 +++
.../tez/shufflehandler/FadvisedChunkedFile.java | 78 ++
.../tez/shufflehandler/FadvisedFileRegion.java | 160 ++++
.../apache/tez/shufflehandler/IndexCache.java | 199 +++++
.../tez/shufflehandler/ShuffleHandler.java | 840 +++++++++++++++++++
.../tez/tests/TestExternalTezServices.java | 183 ++++
.../org/apache/tez/util/ProtoConverters.java | 172 ++++
.../src/test/proto/TezDaemonProtocol.proto | 84 ++
.../src/test/resources/log4j.properties | 19 +
.../org/apache/tez/runtime/task/TezChild.java | 2 +-
.../apache/tez/runtime/task/TezTaskRunner.java | 2 +-
31 files changed, 3943 insertions(+), 11 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index d7e4be5..975ce65 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -1,5 +1,6 @@
ALL CHANGES:
TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration.
TEZ-2006. Task communication plane needs to be pluggable.
+ TEZ-2090. Add tests for jobs running in external services.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a0b82e4..2ddb113 100644
--- a/pom.xml
+++ b/pom.xml
@@ -175,6 +175,11 @@
<type>test-jar</type>
</dependency>
<dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-ext-service-tests</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
<groupId>org.apache.pig</groupId>
<artifactId>pig</artifactId>
<version>${pig.version}</version>
@@ -680,6 +685,7 @@
<module>tez-ui</module>
<module>tez-plugins</module>
<module>tez-tools</module>
+ <module>tez-ext-service-tests</module>
<module>tez-dist</module>
<module>docs</module>
</modules>
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 5398a90..34fdb15 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -1219,6 +1219,8 @@ public class TezConfiguration extends Configuration {
public static final String TEZ_AM_CONTAINER_LAUNCHER_CLASS = TEZ_AM_PREFIX + "container-launcher.class";
@ConfigurationScope(Scope.VERTEX)
public static final String TEZ_AM_TASK_SCHEDULER_CLASS = TEZ_AM_PREFIX + "task-scheduler.class";
+ @ConfigurationScope(Scope.VERTEX)
+ public static final String TEZ_AM_TASK_COMMUNICATOR_CLASS = TEZ_AM_PREFIX + "task-communicator.class";
// TODO only validate property here, value can also be validated if necessary
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index 97f9c16..c9f85e0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -14,6 +14,7 @@
package org.apache.tez.dag.api;
+import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Map;
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 9b2d889..41675fe 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -44,5 +44,8 @@ public interface TaskCommunicatorContext {
// TODO TEZ-2003 Move to vertex, taskIndex, version
void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
+ // TODO TEZ-2003 Add an API to register task failure - for example, a communication failure.
+ // This will have to take into consideration the TA_FAILED event
+
// TODO Eventually Add methods to report availability stats to the scheduler.
}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index e40f79c..77d2e39 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -74,16 +74,22 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
new ConcurrentHashMap<TaskAttempt, ContainerId>();
private final TezTaskUmbilicalProtocol taskUmbilical;
+ private final String tokenIdentifier;
+ private final Token<JobTokenIdentifier> sessionToken;
private InetSocketAddress address;
private Server server;
- private static final class ContainerInfo {
+ public static final class ContainerInfo {
- ContainerInfo(ContainerId containerId) {
+ ContainerInfo(ContainerId containerId, String host, int port) {
this.containerId = containerId;
+ this.host = host;
+ this.port = port;
}
- ContainerId containerId;
+ final ContainerId containerId;
+ public final String host;
+ public final int port;
TezHeartbeatResponse lastResponse = null;
TaskSpec taskSpec = null;
long lastRequestId = 0;
@@ -110,6 +116,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
super(TezTaskCommunicatorImpl.class.getName());
this.taskCommunicatorContext = taskCommunicatorContext;
this.taskUmbilical = new TezTaskUmbilicalProtocolImpl();
+ this.tokenIdentifier = this.taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString();
+ this.sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
}
@@ -130,9 +138,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
try {
JobTokenSecretManager jobTokenSecretManager =
new JobTokenSecretManager();
- Token<JobTokenIdentifier> sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
- jobTokenSecretManager.addTokenForJob(
- taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString(), sessionToken);
+ jobTokenSecretManager.addTokenForJob(tokenIdentifier, sessionToken);
server = new RPC.Builder(conf)
.setProtocol(TezTaskUmbilicalProtocol.class)
@@ -184,7 +190,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
@Override
public void registerRunningContainer(ContainerId containerId, String host, int port) {
- ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId, new ContainerInfo(containerId));
+ ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId, new ContainerInfo(containerId, host, port));
if (oldInfo != null) {
throw new TezUncheckedException("Multiple registrations for containerId: " + containerId);
}
@@ -232,9 +238,9 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
". Already registered to containerId: " + oldId);
}
}
-
}
+
@Override
public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
TaskAttempt taskAttempt = new TaskAttempt(taskAttemptID);
@@ -260,6 +266,18 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
return address;
}
+ protected String getTokenIdentifier() {
+ return tokenIdentifier;
+ }
+
+ protected Token<JobTokenIdentifier> getSessionToken() {
+ return sessionToken;
+ }
+
+ protected TaskCommunicatorContext getTaskCommunicatorContext() {
+ return taskCommunicatorContext;
+ }
+
public TezTaskUmbilicalProtocol getUmbilical() {
return this.taskUmbilical;
}
@@ -473,4 +491,12 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
return "TaskAttempt{" + "taskAttemptId=" + taskAttemptId + '}';
}
}
+
+ protected ContainerInfo getContainerInfo(ContainerId containerId) {
+ return registeredContainers.get(containerId);
+ }
+
+ protected ContainerId getContainerForAttempt(TezTaskAttemptID taskAttemptId) {
+ return attemptToContainerMap.get(new TaskAttempt(taskAttemptId));
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 982e398..a77b53a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -350,7 +350,7 @@ public class TaskSchedulerEventHandler extends AbstractService
try {
Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
.getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
- Integer.class, String.class, Configuration.class);
+ int.class, String.class, Configuration.class);
ctor.setAccessible(true);
TaskSchedulerService taskSchedulerService =
ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
new file mode 100644
index 0000000..37f68b1
--- /dev/null
+++ b/tez-ext-service-tests/pom.xml
@@ -0,0 +1,161 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>tez</artifactId>
+ <groupId>org.apache.tez</groupId>
+ <version>0.7.0-SNAPSHOT</version>
+ </parent>
+
+ <!-- TODO TEZ-2003 Merge this into the tez-tests module -->
+ <artifactId>tez-ext-service-tests</artifactId>
+
+ <dependencies>
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-runtime-internals</artifactId>
+ </dependency>
+ <dependency>
+ <!-- Required for the ShuffleHandler -->
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-runtime-library</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-dag</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-tests</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-server-tests</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ </dependencies>
+
+ <build>
+ <!--
+ Include all files in src/main/resources. By default, do not apply property
+ substitution (filtering=false), but do apply property substitution to
+ version-info.properties (filtering=true). This will substitute the
+ version information correctly, but prevent Maven from altering other files.
+ -->
+ <resources>
+ <resource>
+ <directory>${basedir}/src/main/resources</directory>
+ <excludes>
+ <exclude>tez-api-version-info.properties</exclude>
+ </excludes>
+ <filtering>false</filtering>
+ </resource>
+ <resource>
+ <directory>${basedir}/src/main/resources</directory>
+ <includes>
+ <include>tez-api-version-info.properties</include>
+ </includes>
+ <filtering>true</filtering>
+ </resource>
+ </resources>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.rat</groupId>
+ <artifactId>apache-rat-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-maven-plugins</artifactId>
+ <executions>
+ <execution>
+ <id>compile-protoc</id>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>protoc</goal>
+ </goals>
+ <configuration>
+ <protocVersion>${protobuf.version}</protocVersion>
+ <protocCommand>${protoc.path}</protocCommand>
+ <imports>
+ <param>${basedir}/src/test/proto</param>
+ <param>${basedir}/../tez-api/src/main/proto</param>
+ </imports>
+ <source>
+ <directory>${basedir}/src/test/proto</directory>
+ <includes>
+ <include>TezDaemonProtocol.proto</include>
+ </includes>
+ </source>
+ <output>${project.build.directory}/generated-test-sources/java</output>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+</project>
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
new file mode 100644
index 0000000..ac50878
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
@@ -0,0 +1,152 @@
+/*
+ * 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.tez.dag.app;
+
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.protobuf.Message;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.service.impl.TezTestServiceProtocolClientImpl;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+
+public class TezTestServiceCommunicator extends AbstractService {
+
+ private final ConcurrentMap<String, TezTestServiceProtocolBlockingPB> hostProxies;
+ private final ListeningExecutorService executor;
+
+ // TODO Convert this into a singleton
+ public TezTestServiceCommunicator(int numThreads) {
+ super(TezTestServiceCommunicator.class.getSimpleName());
+ ExecutorService localExecutor = Executors.newFixedThreadPool(numThreads,
+ new ThreadFactoryBuilder().setNameFormat("TezTestServiceCommunicator #%2d").build());
+ this.hostProxies = new ConcurrentHashMap<String, TezTestServiceProtocolBlockingPB>();
+ executor = MoreExecutors.listeningDecorator(localExecutor);
+ }
+
+ @Override
+ public void serviceStop() {
+ executor.shutdownNow();
+ }
+
+
+ public void runContainer(RunContainerRequestProto request, String host, int port,
+ final ExecuteRequestCallback<RunContainerResponseProto> callback) {
+ ListenableFuture<RunContainerResponseProto> future = executor.submit(new RunContainerCallable(request, host, port));
+ Futures.addCallback(future, new FutureCallback<RunContainerResponseProto>() {
+ @Override
+ public void onSuccess(RunContainerResponseProto result) {
+ callback.setResponse(result);
+ }
+
+ @Override
+ public void onFailure(Throwable t) {
+ callback.indicateError(t);
+ }
+ });
+
+ }
+
+ public void submitWork(SubmitWorkRequestProto request, String host, int port,
+ final ExecuteRequestCallback<SubmitWorkResponseProto> callback) {
+ ListenableFuture<SubmitWorkResponseProto> future = executor.submit(new SubmitWorkCallable(request, host, port));
+ Futures.addCallback(future, new FutureCallback<SubmitWorkResponseProto>() {
+ @Override
+ public void onSuccess(SubmitWorkResponseProto result) {
+ callback.setResponse(result);
+ }
+
+ @Override
+ public void onFailure(Throwable t) {
+ callback.indicateError(t);
+ }
+ });
+
+ }
+
+
+ private class RunContainerCallable implements Callable<RunContainerResponseProto> {
+
+ final String hostname;
+ final int port;
+ final RunContainerRequestProto request;
+
+ private RunContainerCallable(RunContainerRequestProto request, String hostname, int port) {
+ this.hostname = hostname;
+ this.port = port;
+ this.request = request;
+ }
+
+ @Override
+ public RunContainerResponseProto call() throws Exception {
+ return getProxy(hostname, port).runContainer(null, request);
+ }
+ }
+
+ private class SubmitWorkCallable implements Callable<SubmitWorkResponseProto> {
+ final String hostname;
+ final int port;
+ final SubmitWorkRequestProto request;
+
+ private SubmitWorkCallable(SubmitWorkRequestProto request, String hostname, int port) {
+ this.hostname = hostname;
+ this.port = port;
+ this.request = request;
+ }
+
+ @Override
+ public SubmitWorkResponseProto call() throws Exception {
+ return getProxy(hostname, port).submitWork(null, request);
+ }
+ }
+
+ public interface ExecuteRequestCallback<T extends Message> {
+ void setResponse(T response);
+ void indicateError(Throwable t);
+ }
+
+ private TezTestServiceProtocolBlockingPB getProxy(String hostname, int port) {
+ String hostId = getHostIdentifier(hostname, port);
+
+ TezTestServiceProtocolBlockingPB proxy = hostProxies.get(hostId);
+ if (proxy == null) {
+ proxy = new TezTestServiceProtocolClientImpl(getConfig(), hostname, port);
+ TezTestServiceProtocolBlockingPB proxyOld = hostProxies.putIfAbsent(hostId, proxy);
+ if (proxyOld != null) {
+ // TODO Shutdown the new proxy.
+ proxy = proxyOld;
+ }
+ }
+ return proxy;
+ }
+
+ private String getHostIdentifier(String hostname, int port) {
+ return hostname + ":" + port;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
new file mode 100644
index 0000000..e83165b
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -0,0 +1,144 @@
+/*
+ * 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.tez.dag.app.launcher;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TezTestServiceCommunicator;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunchFailed;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+import org.apache.tez.service.TezTestServiceConfConstants;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+
+public class TezTestServiceContainerLauncher extends AbstractService implements ContainerLauncher {
+
+ // TODO Support interruptability of tasks which haven't yet been launched.
+
+ // TODO May need multiple connections per target machine, depending upon how synchronization is handled in the RPC layer
+
+ static final Log LOG = LogFactory.getLog(TezTestServiceContainerLauncher.class);
+
+ private final AppContext context;
+ private final String tokenIdentifier;
+ private final TaskAttemptListener tal;
+ private final int servicePort;
+ private final TezTestServiceCommunicator communicator;
+ private final Clock clock;
+
+
+ // Configuration passed in here to set up final parameters
+ public TezTestServiceContainerLauncher(AppContext appContext, Configuration conf,
+ TaskAttemptListener tal) {
+ super(TezTestServiceContainerLauncher.class.getName());
+ this.clock = appContext.getClock();
+ int numThreads = conf.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS,
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT);
+
+ this.servicePort = conf.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT, -1);
+ Preconditions.checkArgument(servicePort > 0,
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT + " must be set");
+ this.communicator = new TezTestServiceCommunicator(numThreads);
+ this.context = appContext;
+ this.tokenIdentifier = context.getApplicationID().toString();
+ this.tal = tal;
+ }
+
+ @Override
+ public void serviceInit(Configuration conf) {
+ communicator.init(conf);
+ }
+
+ @Override
+ public void serviceStart() {
+ communicator.start();
+ }
+
+ @Override
+ public void serviceStop() {
+ communicator.stop();
+ }
+
+ @Override
+ public void handle(NMCommunicatorEvent event) {
+ switch (event.getType()) {
+ case CONTAINER_LAUNCH_REQUEST:
+ final NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
+ RunContainerRequestProto runRequest = constructRunContainerRequest(launchEvent);
+ communicator.runContainer(runRequest, launchEvent.getNodeId().getHost(),
+ launchEvent.getNodeId().getPort(),
+ new TezTestServiceCommunicator.ExecuteRequestCallback<TezTestServiceProtocolProtos.RunContainerResponseProto>() {
+ @Override
+ public void setResponse(TezTestServiceProtocolProtos.RunContainerResponseProto response) {
+ LOG.info("Container: " + launchEvent.getContainerId() + " launch succeeded on host: " + launchEvent.getNodeId());
+ context.getEventHandler().handle(new AMContainerEventLaunched(launchEvent.getContainerId()));
+ ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
+ launchEvent.getContainerId(), clock.getTime(), context.getApplicationAttemptId());
+ context.getHistoryHandler().handle(new DAGHistoryEvent(
+ null, lEvt));
+ }
+
+ @Override
+ public void indicateError(Throwable t) {
+ LOG.error("Failed to launch container: " + launchEvent.getContainer() + " on host: " + launchEvent.getNodeId(), t);
+ sendContainerLaunchFailedMsg(launchEvent.getContainerId(), t);
+ }
+ });
+ break;
+ case CONTAINER_STOP_REQUEST:
+ LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + event);
+ // that the container is actually done (normally received from RM)
+ // TODO Sending this out for an un-launched container is invalid
+ context.getEventHandler().handle(new AMContainerEvent(event.getContainerId(),
+ AMContainerEventType.C_NM_STOP_SENT));
+ break;
+ }
+ }
+
+ private RunContainerRequestProto constructRunContainerRequest(NMCommunicatorLaunchRequestEvent event) {
+ RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
+ builder.setAmHost(tal.getAddress().getHostName()).setAmPort(tal.getAddress().getPort());
+ builder.setAppAttemptNumber(event.getContainer().getId().getApplicationAttemptId().getAttemptId());
+ builder.setApplicationIdString(
+ event.getContainer().getId().getApplicationAttemptId().getApplicationId().toString());
+ builder.setTokenIdentifier(tokenIdentifier);
+ builder.setContainerIdString(event.getContainer().getId().toString());
+ builder.setCredentialsBinary(
+ ByteString.copyFrom(event.getContainerLaunchContext().getTokens()));
+ // TODO Avoid reading this from the environment
+ builder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
+ return builder.build();
+ }
+
+ @SuppressWarnings("unchecked")
+ void sendContainerLaunchFailedMsg(ContainerId containerId, Throwable t) {
+ context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, t == null ? "" : t.getMessage()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
new file mode 100644
index 0000000..8c8e486
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
@@ -0,0 +1,66 @@
+/*
+ * 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.tez.dag.app.launcher;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+
+public class TezTestServiceNoOpContainerLauncher extends AbstractService implements ContainerLauncher {
+
+ static final Log LOG = LogFactory.getLog(TezTestServiceNoOpContainerLauncher.class);
+
+ private final AppContext context;
+ private final Clock clock;
+
+ public TezTestServiceNoOpContainerLauncher(AppContext appContext, Configuration conf,
+ TaskAttemptListener tal) {
+ super(TezTestServiceNoOpContainerLauncher.class.getName());
+ this.context = appContext;
+ this.clock = appContext.getClock();
+ }
+
+ @Override
+ public void handle(NMCommunicatorEvent event) {
+ switch(event.getType()) {
+ case CONTAINER_LAUNCH_REQUEST:
+ final NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
+ LOG.info("No-op launch for container: " + launchEvent.getContainerId() + " succeeded on host: " + launchEvent.getNodeId());
+ context.getEventHandler().handle(new AMContainerEventLaunched(launchEvent.getContainerId()));
+ ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
+ launchEvent.getContainerId(), clock.getTime(), context.getApplicationAttemptId());
+ context.getHistoryHandler().handle(new DAGHistoryEvent(
+ null, lEvt));
+ break;
+ case CONTAINER_STOP_REQUEST:
+ LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + event);
+ context.getEventHandler().handle(new AMContainerEvent(event.getContainerId(),
+ AMContainerEventType.C_NM_STOP_SENT));
+ break;
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
new file mode 100644
index 0000000..e3c18bf
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -0,0 +1,347 @@
+/*
+ * 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.tez.dag.app.rm;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.service.TezTestServiceConfConstants;
+
+
+// TODO Registration with RM - so that the AM is considered dead and restarted in the expiry interval - 10 minutes.
+
+public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
+
+ private static final Log LOG = LogFactory.getLog(TezTestServiceTaskSchedulerService.class);
+
+ private final ExecutorService appCallbackExecutor;
+ private final TaskSchedulerAppCallback appClientDelegate;
+ private final AppContext appContext;
+ private final List<String> serviceHosts;
+ private final ContainerFactory containerFactory;
+ private final Random random = new Random();
+ // Currently all services must be running on the same port.
+ private final int containerPort;
+
+ private final String clientHostname;
+ private final int clientPort;
+ private final String trackingUrl;
+ private final AtomicBoolean isStopped = new AtomicBoolean(false);
+ private final ConcurrentMap<Object, ContainerId> runningTasks =
+ new ConcurrentHashMap<Object, ContainerId>();
+
+ private final AMRMClientAsync<AMRMClient.ContainerRequest> amRmClient;
+
+ // Per instance
+ private final int memoryPerInstance;
+ private final int coresPerInstance;
+ private final int executorsPerInstance;
+
+ // Per Executor Thread
+ private final Resource resourcePerContainer;
+
+
+ public TezTestServiceTaskSchedulerService(TaskSchedulerAppCallback appClient,
+ AppContext appContext,
+ String clientHostname, int clientPort,
+ String trackingUrl,
+ Configuration conf) {
+ // Accepting configuration here to allow setting up fields as final
+ super(TezTestServiceTaskSchedulerService.class.getName());
+ this.appCallbackExecutor = createAppCallbackExecutorService();
+ this.appClientDelegate = createAppCallbackDelegate(appClient);
+ this.appContext = appContext;
+ this.serviceHosts = new LinkedList<String>();
+ this.containerFactory = new ContainerFactory(appContext);
+
+ this.memoryPerInstance = conf
+ .getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, -1);
+ Preconditions.checkArgument(memoryPerInstance > 0,
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB +
+ " must be configured");
+
+ this.executorsPerInstance = conf.getInt(
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE,
+ -1);
+ Preconditions.checkArgument(executorsPerInstance > 0,
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE +
+ " must be configured");
+
+ this.coresPerInstance = conf
+ .getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_VCPUS_PER_INSTANCE,
+ executorsPerInstance);
+
+ this.containerPort = conf.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT, -1);
+ Preconditions.checkArgument(executorsPerInstance > 0,
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT + " must be configured");
+
+ this.clientHostname = clientHostname;
+ this.clientPort = clientPort;
+ this.trackingUrl = trackingUrl;
+
+ int memoryPerContainer = (int) (memoryPerInstance / (float) executorsPerInstance);
+ int coresPerContainer = (int) (coresPerInstance / (float) executorsPerInstance);
+ this.resourcePerContainer = Resource.newInstance(memoryPerContainer, coresPerContainer);
+ this.amRmClient = TezAMRMClientAsync.createAMRMClientAsync(5000, new FakeAmRmCallbackHandler());
+
+ String[] hosts = conf.getTrimmedStrings(TezTestServiceConfConstants.TEZ_TEST_SERVICE_HOSTS);
+ if (hosts == null || hosts.length == 0) {
+ hosts = new String[]{"localhost"};
+ }
+ for (String host : hosts) {
+ serviceHosts.add(host);
+ }
+
+ LOG.info("Running with configuration: " +
+ "memoryPerInstance=" + memoryPerInstance +
+ ", vcoresPerInstance=" + coresPerInstance +
+ ", executorsPerInstance=" + executorsPerInstance +
+ ", resourcePerContainerInferred=" + resourcePerContainer +
+ ", hosts=" + serviceHosts.toString());
+
+ }
+
+ @Override
+ public void serviceInit(Configuration conf) {
+ amRmClient.init(conf);
+ }
+
+ @Override
+ public void serviceStart() {
+ amRmClient.start();
+ RegisterApplicationMasterResponse response;
+ try {
+ amRmClient.registerApplicationMaster(clientHostname, clientPort, trackingUrl);
+ } catch (YarnException e) {
+ throw new TezUncheckedException(e);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
+ }
+
+ @Override
+ public void serviceStop() {
+ if (!this.isStopped.getAndSet(true)) {
+
+ try {
+ TaskSchedulerAppCallback.AppFinalStatus status = appClientDelegate.getFinalAppStatus();
+ amRmClient.unregisterApplicationMaster(status.exitStatus, status.exitMessage,
+ status.postCompletionTrackingUrl);
+ } catch (YarnException e) {
+ throw new TezUncheckedException(e);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
+ appCallbackExecutor.shutdownNow();
+ }
+ }
+
+ @Override
+ public Resource getAvailableResources() {
+ // TODO This needs information about all running executors, and the amount of memory etc available across the cluster.
+ return Resource
+ .newInstance(Ints.checkedCast(serviceHosts.size() * memoryPerInstance),
+ serviceHosts.size() * coresPerInstance);
+ }
+
+ @Override
+ public int getClusterNodeCount() {
+ return serviceHosts.size();
+ }
+
+ @Override
+ public void resetMatchLocalityForAllHeldContainers() {
+ }
+
+ @Override
+ public Resource getTotalResources() {
+ return Resource
+ .newInstance(Ints.checkedCast(serviceHosts.size() * memoryPerInstance),
+ serviceHosts.size() * coresPerInstance);
+ }
+
+ @Override
+ public void blacklistNode(NodeId nodeId) {
+ LOG.info("DEBUG: BlacklistNode not supported");
+ }
+
+ @Override
+ public void unblacklistNode(NodeId nodeId) {
+ LOG.info("DEBUG: unBlacklistNode not supported");
+ }
+
+ @Override
+ public void allocateTask(Object task, Resource capability, String[] hosts, String[] racks,
+ Priority priority, Object containerSignature, Object clientCookie) {
+ String host = selectHost(hosts);
+ Container container =
+ containerFactory.createContainer(resourcePerContainer, priority, host, containerPort);
+ runningTasks.put(task, container.getId());
+ appClientDelegate.taskAllocated(task, clientCookie, container);
+ }
+
+
+ @Override
+ public void allocateTask(Object task, Resource capability, ContainerId containerId,
+ Priority priority, Object containerSignature, Object clientCookie) {
+ String host = selectHost(null);
+ Container container =
+ containerFactory.createContainer(resourcePerContainer, priority, host, containerPort);
+ runningTasks.put(task, container.getId());
+ appClientDelegate.taskAllocated(task, clientCookie, container);
+ }
+
+ @Override
+ public boolean deallocateTask(Object task, boolean taskSucceeded) {
+ ContainerId containerId = runningTasks.remove(task);
+ if (containerId == null) {
+ LOG.error("Could not determine ContainerId for task: " + task +
+ " . Could have hit a race condition. Ignoring." +
+ " The query may hang since this \"unknown\" container is now taking up a slot permanently");
+ return false;
+ }
+ appClientDelegate.containerBeingReleased(containerId);
+ return true;
+ }
+
+ @Override
+ public Object deallocateContainer(ContainerId containerId) {
+ LOG.info("DEBUG: Ignoring deallocateContainer for containerId: " + containerId);
+ return null;
+ }
+
+ @Override
+ public void setShouldUnregister() {
+
+ }
+
+ @Override
+ public boolean hasUnregistered() {
+ // Nothing to do. No registration involved.
+ return true;
+ }
+
+ private ExecutorService createAppCallbackExecutorService() {
+ return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+ .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
+ }
+
+ private TaskSchedulerAppCallback createAppCallbackDelegate(
+ TaskSchedulerAppCallback realAppClient) {
+ return new TaskSchedulerAppCallbackWrapper(realAppClient,
+ appCallbackExecutor);
+ }
+
+ private String selectHost(String[] requestedHosts) {
+ String host = null;
+ if (requestedHosts != null && requestedHosts.length > 0) {
+ Arrays.sort(requestedHosts);
+ host = requestedHosts[0];
+ LOG.info("Selected host: " + host + " from requested hosts: " + Arrays.toString(requestedHosts));
+ } else {
+ host = serviceHosts.get(random.nextInt(serviceHosts.size()));
+ LOG.info("Selected random host: " + host + " since the request contained no host information");
+ }
+ return host;
+ }
+
+ static class ContainerFactory {
+ final AppContext appContext;
+ AtomicInteger nextId;
+
+ public ContainerFactory(AppContext appContext) {
+ this.appContext = appContext;
+ this.nextId = new AtomicInteger(2);
+ }
+
+ public Container createContainer(Resource capability, Priority priority, String hostname, int port) {
+ ApplicationAttemptId appAttemptId = appContext.getApplicationAttemptId();
+ ContainerId containerId = ContainerId.newInstance(appAttemptId, nextId.getAndIncrement());
+ NodeId nodeId = NodeId.newInstance(hostname, port);
+ String nodeHttpAddress = "hostname:0";
+
+ Container container = Container.newInstance(containerId,
+ nodeId,
+ nodeHttpAddress,
+ capability,
+ priority,
+ null);
+
+ return container;
+ }
+ }
+
+ private static class FakeAmRmCallbackHandler implements AMRMClientAsync.CallbackHandler {
+
+ @Override
+ public void onContainersCompleted(List<ContainerStatus> statuses) {
+
+ }
+
+ @Override
+ public void onContainersAllocated(List<Container> containers) {
+
+ }
+
+ @Override
+ public void onShutdownRequest() {
+
+ }
+
+ @Override
+ public void onNodesUpdated(List<NodeReport> updatedNodes) {
+
+ }
+
+ @Override
+ public float getProgress() {
+ return 0;
+ }
+
+ @Override
+ public void onError(Throwable e) {
+
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
new file mode 100644
index 0000000..78cdcde
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -0,0 +1,182 @@
+/*
+ * 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.tez.dag.app.taskcomm;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.protobuf.ByteString;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
+import org.apache.tez.dag.app.TezTestServiceCommunicator;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+import org.apache.tez.util.ProtoConverters;
+
+
+public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl {
+
+ private static final Log LOG = LogFactory.getLog(TezTestServiceTaskCommunicatorImpl.class);
+
+ private final TezTestServiceCommunicator communicator;
+ private final SubmitWorkRequestProto BASE_SUBMIT_WORK_REQUEST;
+ private final ConcurrentMap<String, ByteBuffer> credentialMap;
+
+ public TezTestServiceTaskCommunicatorImpl(
+ TaskCommunicatorContext taskCommunicatorContext) {
+ super(taskCommunicatorContext);
+ // TODO Maybe make this configurable
+ this.communicator = new TezTestServiceCommunicator(3);
+
+ SubmitWorkRequestProto.Builder baseBuilder = SubmitWorkRequestProto.newBuilder();
+
+ // TODO Avoid reading this from the environment
+ baseBuilder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
+ baseBuilder.setApplicationIdString(
+ taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString());
+ baseBuilder
+ .setAppAttemptNumber(taskCommunicatorContext.getApplicationAttemptId().getAttemptId());
+ baseBuilder.setTokenIdentifier(getTokenIdentifier());
+
+ BASE_SUBMIT_WORK_REQUEST = baseBuilder.build();
+
+ credentialMap = new ConcurrentHashMap<String, ByteBuffer>();
+ }
+
+ @Override
+ public void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
+ this.communicator.init(conf);
+ }
+
+ @Override
+ public void serviceStart() {
+ super.serviceStart();
+ this.communicator.start();
+ }
+
+ @Override
+ public void serviceStop() {
+ super.serviceStop();
+ }
+
+
+ @Override
+ public void registerRunningContainer(ContainerId containerId, String hostname, int port) {
+ super.registerRunningContainer(containerId, hostname, port);
+ }
+
+ @Override
+ public void registerContainerEnd(ContainerId containerId) {
+ super.registerContainerEnd(containerId);
+ }
+
+ @Override
+ public void registerRunningTaskAttempt(final ContainerId containerId, final TaskSpec taskSpec,
+ Map<String, LocalResource> additionalResources,
+ Credentials credentials,
+ boolean credentialsChanged) {
+ super.registerRunningTaskAttempt(containerId, taskSpec, additionalResources, credentials,
+ credentialsChanged);
+ SubmitWorkRequestProto requestProto = null;
+ try {
+ requestProto = constructSubmitWorkRequest(containerId, taskSpec);
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to construct request", e);
+ }
+ ContainerInfo containerInfo = getContainerInfo(containerId);
+ String host;
+ int port;
+ if (containerInfo != null) {
+ synchronized (containerInfo) {
+ host = containerInfo.host;
+ port = containerInfo.port;
+ }
+ } else {
+ // TODO Handle this properly
+ throw new RuntimeException("ContainerInfo not found for container: " + containerId +
+ ", while trying to launch task: " + taskSpec.getTaskAttemptID());
+ }
+ communicator.submitWork(requestProto, host, port,
+ new TezTestServiceCommunicator.ExecuteRequestCallback<SubmitWorkResponseProto>() {
+ @Override
+ public void setResponse(SubmitWorkResponseProto response) {
+ LOG.info("Successfully launched task: " + taskSpec.getTaskAttemptID());
+ getTaskCommunicatorContext()
+ .taskStartedRemotely(taskSpec.getTaskAttemptID(), containerId);
+ }
+
+ @Override
+ public void indicateError(Throwable t) {
+ // TODO Handle this error. This is where an API on the context to indicate failure / rejection comes in.
+ LOG.info("Failed to run task: " + taskSpec.getTaskAttemptID() + " on containerId: " +
+ containerId, t);
+ }
+ });
+ }
+
+ @Override
+ public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
+ super.unregisterRunningTaskAttempt(taskAttemptID);
+ // Nothing else to do for now. The push API in the test does not support termination of a running task
+ }
+
+ private SubmitWorkRequestProto constructSubmitWorkRequest(ContainerId containerId,
+ TaskSpec taskSpec) throws
+ IOException {
+ SubmitWorkRequestProto.Builder builder =
+ SubmitWorkRequestProto.newBuilder(BASE_SUBMIT_WORK_REQUEST);
+ builder.setContainerIdString(containerId.toString());
+ builder.setAmHost(getAddress().getHostName());
+ builder.setAmPort(getAddress().getPort());
+ Credentials taskCredentials = new Credentials();
+ // Credentials can change across DAGs. Ideally construct only once per DAG.
+ taskCredentials.addAll(getTaskCommunicatorContext().getCredentials());
+
+ ByteBuffer credentialsBinary = credentialMap.get(taskSpec.getDAGName());
+ if (credentialsBinary == null) {
+ credentialsBinary = serializeCredentials(getTaskCommunicatorContext().getCredentials());
+ credentialMap.putIfAbsent(taskSpec.getDAGName(), credentialsBinary.duplicate());
+ } else {
+ credentialsBinary = credentialsBinary.duplicate();
+ }
+ builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
+ builder.setTaskSpec(ProtoConverters.convertTaskSpecToProto(taskSpec));
+ return builder.build();
+ }
+
+ private ByteBuffer serializeCredentials(Credentials credentials) throws IOException {
+ Credentials containerCredentials = new Credentials();
+ containerCredentials.addAll(credentials);
+ DataOutputBuffer containerTokens_dob = new DataOutputBuffer();
+ containerCredentials.writeTokenStorageToStream(containerTokens_dob);
+ ByteBuffer containerCredentialsBuffer = ByteBuffer.wrap(containerTokens_dob.getData(), 0,
+ containerTokens_dob.getLength());
+ return containerCredentialsBuffer;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
new file mode 100644
index 0000000..2bca4ed
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
@@ -0,0 +1,27 @@
+/*
+ * 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.tez.service;
+
+import java.io.IOException;
+
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+
+public interface ContainerRunner {
+
+ void queueContainer(RunContainerRequestProto request) throws IOException;
+ void submitWork(SubmitWorkRequestProto request) throws IOException;
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
new file mode 100644
index 0000000..f47bd67
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
@@ -0,0 +1,163 @@
+/*
+ * 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.tez.service;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.tez.service.impl.TezTestService;
+
+public class MiniTezTestServiceCluster extends AbstractService {
+
+ private static final Log LOG = LogFactory.getLog(MiniTezTestServiceCluster.class);
+
+ private final File testWorkDir;
+ private final long availableMemory;
+ private final int numExecutorsPerService;
+ private final String[] localDirs;
+ private final Configuration clusterSpecificConfiguration = new Configuration(false);
+
+ private TezTestService tezTestService;
+
+ public static MiniTezTestServiceCluster create(String clusterName, int numExecutorsPerService, long availableMemory, int numLocalDirs) {
+ return new MiniTezTestServiceCluster(clusterName, numExecutorsPerService, availableMemory, numLocalDirs);
+ }
+
+ // TODO Add support for multiple instances
+ private MiniTezTestServiceCluster(String clusterName, int numExecutorsPerService, long availableMemory, int numLocalDirs) {
+ super(clusterName + "_TezTestServerCluster");
+ Preconditions.checkArgument(numExecutorsPerService > 0);
+ Preconditions.checkArgument(availableMemory > 0);
+ Preconditions.checkArgument(numLocalDirs > 0);
+ String clusterNameTrimmed = clusterName.replace("$", "") + "_TezTestServerCluster";
+ File targetWorkDir = new File("target", clusterNameTrimmed);
+ try {
+ FileContext.getLocalFSFileContext().delete(
+ new Path(targetWorkDir.getAbsolutePath()), true);
+ } catch (Exception e) {
+ LOG.warn("Could not cleanup test workDir: " + targetWorkDir, e);
+ throw new RuntimeException("Could not cleanup test workDir: " + targetWorkDir, e);
+ }
+
+ if (Shell.WINDOWS) {
+ // The test working directory can exceed the maximum path length supported
+ // by some Windows APIs and cmd.exe (260 characters). To work around this,
+ // create a symlink in temporary storage with a much shorter path,
+ // targeting the full path to the test working directory. Then, use the
+ // symlink as the test working directory.
+ String targetPath = targetWorkDir.getAbsolutePath();
+ File link = new File(System.getProperty("java.io.tmpdir"),
+ String.valueOf(System.currentTimeMillis()));
+ String linkPath = link.getAbsolutePath();
+
+ try {
+ FileContext.getLocalFSFileContext().delete(new Path(linkPath), true);
+ } catch (IOException e) {
+ throw new YarnRuntimeException("could not cleanup symlink: " + linkPath, e);
+ }
+
+ // Guarantee target exists before creating symlink.
+ targetWorkDir.mkdirs();
+
+ Shell.ShellCommandExecutor shexec = new Shell.ShellCommandExecutor(
+ Shell.getSymlinkCommand(targetPath, linkPath));
+ try {
+ shexec.execute();
+ } catch (IOException e) {
+ throw new YarnRuntimeException(String.format(
+ "failed to create symlink from %s to %s, shell output: %s", linkPath,
+ targetPath, shexec.getOutput()), e);
+ }
+
+ this.testWorkDir = link;
+ } else {
+ this.testWorkDir = targetWorkDir;
+ }
+ this.numExecutorsPerService = numExecutorsPerService;
+ this.availableMemory = availableMemory;
+
+ // Setup Local Dirs
+ localDirs = new String[numLocalDirs];
+ for (int i = 0 ; i < numLocalDirs ; i++) {
+ File f = new File(testWorkDir, "localDir");
+ f.mkdirs();
+ LOG.info("Created localDir: " + f.getAbsolutePath());
+ localDirs[i] = f.getAbsolutePath();
+ }
+ }
+
+ @Override
+ public void serviceInit(Configuration conf) {
+ tezTestService = new TezTestService(conf, numExecutorsPerService, availableMemory, localDirs);
+ tezTestService.init(conf);
+
+ }
+
+ @Override
+ public void serviceStart() {
+ tezTestService.start();
+
+ clusterSpecificConfiguration.set(TezTestServiceConfConstants.TEZ_TEST_SERVICE_HOSTS,
+ getServiceAddress().getHostName());
+ clusterSpecificConfiguration.setInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT,
+ getServiceAddress().getPort());
+
+ clusterSpecificConfiguration.setInt(
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE,
+ numExecutorsPerService);
+ clusterSpecificConfiguration.setLong(
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, availableMemory);
+ }
+
+ @Override
+ public void serviceStop() {
+ tezTestService.stop();
+ }
+
+ /**
+ * return the address at which the service is listening
+ * @return host:port
+ */
+ public InetSocketAddress getServiceAddress() {
+ Preconditions.checkState(getServiceState() == STATE.STARTED);
+ return tezTestService.getListenerAddress();
+ }
+
+ public int getShufflePort() {
+ Preconditions.checkState(getServiceState() == STATE.STARTED);
+ return tezTestService.getShufflePort();
+ }
+
+ public Configuration getClusterSpecificConfiguration() {
+ Preconditions.checkState(getServiceState() == STATE.STARTED);
+ return clusterSpecificConfiguration;
+ }
+
+ // Mainly for verification
+ public int getNumSubmissions() {
+ return tezTestService.getNumSubmissions();
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
new file mode 100644
index 0000000..bf4a5bd
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
@@ -0,0 +1,41 @@
+/*
+ * 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.tez.service;
+
+public class TezTestServiceConfConstants {
+
+ private static final String TEZ_TEST_SERVICE_PREFIX = "tez.test.service.";
+
+ /** Number of executors per instance - used by the scheduler */
+ public static final String TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE = TEZ_TEST_SERVICE_PREFIX + "num.executors.per-instance";
+
+ /** Memory available per instance - used by the scheduler */
+ public static final String TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB = TEZ_TEST_SERVICE_PREFIX + "memory.per.instance.mb";
+
+ /** CPUs available per instance - used by the scheduler */
+ public static final String TEZ_TEST_SERVICE_VCPUS_PER_INSTANCE = TEZ_TEST_SERVICE_PREFIX + "vcpus.per.instance";
+
+
+ /** Hosts on which the service is running. Currently assuming a single port for all instances */
+ public static final String TEZ_TEST_SERVICE_HOSTS = TEZ_TEST_SERVICE_PREFIX + "hosts";
+
+ /** Port on which the Service(s) listen. Current a single port for all instances */
+ public static final String TEZ_TEST_SERVICE_RPC_PORT = TEZ_TEST_SERVICE_PREFIX + "rpc.port";
+
+ /** Number of threads to use in the AM to communicate with the external service */
+ public static final String TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS = TEZ_TEST_SERVICE_PREFIX + "communicator.num.threads";
+ public static final int TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT = 2;
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
new file mode 100644
index 0000000..1108f72
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
@@ -0,0 +1,22 @@
+/*
+ * 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.tez.service;
+
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+
+@ProtocolInfo(protocolName = "org.apache.tez.service.TezTestServiceProtocolBlockingPB", protocolVersion = 1)
+public interface TezTestServiceProtocolBlockingPB extends TezTestServiceProtocolProtos.TezTestServiceProtocol.BlockingInterface {
+}
\ No newline at end of file
[14/51] [abbrv] tez git commit: TEZ-2138. Fix minor bugs in adding
default scheduler, getting launchers. (sseth)
Posted by ss...@apache.org.
TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/5150c203
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/5150c203
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/5150c203
Branch: refs/heads/TEZ-2003
Commit: 5150c2034e9833095936bc1972a4d98d009b71a3
Parents: 8b79df5
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Feb 23 20:53:24 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../src/main/java/org/apache/tez/dag/app/DAGAppMaster.java | 2 +-
.../java/org/apache/tez/dag/app/dag/impl/VertexImpl.java | 4 ++++
.../org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java | 1 +
.../dag/app/launcher/TezTestServiceContainerLauncher.java | 8 +++++---
5 files changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/5150c203/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 4b0a12b..4377f57 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -7,5 +7,6 @@ ALL CHANGES:
TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
TEZ-2131. Add additional tests for tasks running in the AM.
+ TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/5150c203/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 43f8794..6ab0f8e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -1547,7 +1547,7 @@ public class DAGAppMaster extends AbstractService {
@Override
public Integer getContainerLauncherIdentifier(String name) {
- return taskCommunicators.get(name);
+ return containerLaunchers.get(name);
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/5150c203/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index 7733ef9..bdab984 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -1003,6 +1003,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
.get(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, tezDefaultComponentName);
String containerLauncherName = vertexConf
.get(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, tezDefaultComponentName);
+
+ LOG.info("Vertex: " + logIdentifier + " configured with TaskScheduler=" + taskSchedulerName +
+ ", ContainerLauncher=" + containerLauncherName + ", TaskComm=" + taskCommName);
+
taskSchedulerIdentifier = appContext.getTaskScheduerIdentifier(taskSchedulerName);
taskCommunicatorIdentifier = appContext.getTaskCommunicatorIdentifier(taskCommName);
containerLauncherIdentifier = appContext.getContainerLauncherIdentifier(containerLauncherName);
http://git-wip-us.apache.org/repos/asf/tez/blob/5150c203/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index ba46a67..9a8bf16 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -174,6 +174,7 @@ public class TaskSchedulerEventHandler extends AbstractService
for (int i = 0 ; i < schedulerClasses.length ; i++) { // Copy over the rest.
this.taskSchedulerClasses[i] = schedulerClasses[i];
}
+ this.taskSchedulerClasses[foundYarnTaskSchedulerIndex] = TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
} else {
this.taskSchedulerClasses = schedulerClasses;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5150c203/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index 27356bc..cb6308c 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.tez.dag.app.AppContext;
@@ -54,6 +55,7 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
private final int servicePort;
private final TezTestServiceCommunicator communicator;
private final Clock clock;
+ private final ApplicationAttemptId appAttemptId;
// Configuration passed in here to set up final parameters
@@ -70,6 +72,7 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
this.communicator = new TezTestServiceCommunicator(numThreads);
this.context = appContext;
this.tokenIdentifier = context.getApplicationID().toString();
+ this.appAttemptId = appContext.getApplicationAttemptId();
this.tal = tal;
}
@@ -128,9 +131,8 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
InetSocketAddress address = tal.getTaskCommunicator(event.getTaskCommId()).getAddress();
builder.setAmHost(address.getHostName()).setAmPort(address.getPort());
- builder.setAppAttemptNumber(event.getContainer().getId().getApplicationAttemptId().getAttemptId());
- builder.setApplicationIdString(
- event.getContainer().getId().getApplicationAttemptId().getApplicationId().toString());
+ builder.setAppAttemptNumber(appAttemptId.getAttemptId());
+ builder.setApplicationIdString(appAttemptId.getApplicationId().toString());
builder.setTokenIdentifier(tokenIdentifier);
builder.setContainerIdString(event.getContainer().getId().toString());
builder.setCredentialsBinary(
[36/51] [abbrv] tez git commit: TEZ-2004. Define basic interface for
pluggable ContainerLaunchers. (sseth)
Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
index 07dfcd6..25fd13e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
public abstract class TaskSchedulerService extends AbstractService{
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
index 6f897e1..d4cf317 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
index 8ef2a83..cecb019 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
@@ -20,7 +20,7 @@ package org.apache.tez.dag.app.rm.container;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.dag.records.TaskAttemptTerminationCause;
public class AMContainerEventCompleted extends AMContainerEvent {
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index e9e0f04..e63d86d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -28,8 +28,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import org.apache.tez.common.TezUtilsInternal;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.security.Credentials;
@@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
@@ -1068,12 +1067,12 @@ public class AMContainerImpl implements AMContainer {
}
protected void sendStartRequestToNM(ContainerLaunchContext clc) {
- sendEvent(new NMCommunicatorLaunchRequestEvent(clc, container, launcherId, taskCommId));
+ sendEvent(new NMCommunicatorLaunchRequestEvent(clc, container, launcherId, schedulerId, taskCommId));
}
protected void sendStopRequestToNM() {
sendEvent(new NMCommunicatorStopRequestEvent(containerId,
- container.getNodeId(), container.getContainerToken(), launcherId));
+ container.getNodeId(), container.getContainerToken(), launcherId, schedulerId, taskCommId));
}
protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId, TaskAttemptEndReason endReason) {
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 0f35bba..3c3c6a7 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -34,11 +34,14 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -53,23 +56,14 @@ import org.apache.tez.dag.api.TaskHeartbeatRequest;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.app.launcher.ContainerLauncher;
import org.apache.tez.dag.app.launcher.ContainerLauncherRouter;
import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
-import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
-import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
-import org.apache.tez.dag.app.rm.container.AMContainerEventType;
import org.apache.tez.dag.history.HistoryEventHandler;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
-import org.apache.tez.runtime.api.events.DataMovementEvent;
import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
import org.apache.tez.runtime.api.impl.EventMetaData;
-import org.apache.tez.runtime.api.impl.OutputSpec;
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.api.impl.TaskStatistics;
import org.apache.tez.runtime.api.impl.TezEvent;
@@ -89,6 +83,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
public class MockDAGAppMaster extends DAGAppMaster {
private static final Logger LOG = LoggerFactory.getLogger(MockDAGAppMaster.class);
+ ContainerLauncherContext containerLauncherContext;
MockContainerLauncher containerLauncher;
boolean initFailFlag;
boolean startFailFlag;
@@ -121,7 +116,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
// Upon, launch of a container is simulates the container asking for tasks
// Upon receiving a task it simulates completion of the tasks
// It can be used to preempt the container for a given task
- public class MockContainerLauncher extends AbstractService implements ContainerLauncher, Runnable {
+ public class MockContainerLauncher extends ContainerLauncher implements Runnable {
BlockingQueue<NMCommunicatorEvent> eventQueue = new LinkedBlockingQueue<NMCommunicatorEvent>();
Thread eventHandlingThread;
@@ -141,12 +136,14 @@ public class MockDAGAppMaster extends DAGAppMaster {
Map<TezTaskID, Integer> preemptedTasks = Maps.newConcurrentMap();
Map<TezTaskAttemptID, Integer> tasksWithStatusUpdates = Maps.newConcurrentMap();
-
- public MockContainerLauncher(AtomicBoolean goFlag) {
- super("MockContainerLauncher");
+
+ public MockContainerLauncher(AtomicBoolean goFlag,
+ ContainerLauncherContext containerLauncherContext) {
+ super("MockContainerLauncher", containerLauncherContext);
this.goFlag = goFlag;
}
+
public class ContainerData {
ContainerId cId;
TezTaskAttemptID taId;
@@ -211,20 +208,18 @@ public class MockDAGAppMaster extends DAGAppMaster {
executorService.shutdownNow();
}
}
-
+
+
@Override
- public void handle(NMCommunicatorEvent event) {
- switch (event.getType()) {
- case CONTAINER_LAUNCH_REQUEST:
- launch((NMCommunicatorLaunchRequestEvent) event);
- break;
- case CONTAINER_STOP_REQUEST:
- stop((NMCommunicatorStopRequestEvent)event);
- break;
- }
+ public void launchContainer(ContainerLaunchRequest launchRequest) {
+ launch(launchRequest);
}
-
-
+
+ @Override
+ public void stopContainer(ContainerStopRequest stopRequest) {
+ stop(stopRequest);
+ }
+
void waitToGo() {
if (goFlag == null) {
return;
@@ -266,20 +261,19 @@ public class MockDAGAppMaster extends DAGAppMaster {
tasksWithStatusUpdates.put(tId, numUpdates);
}
- void stop(NMCommunicatorStopRequestEvent event) {
+ void stop(ContainerStopRequest event) {
// remove from simulated container list
containers.remove(event.getContainerId());
- getContext().getEventHandler().handle(
- new AMContainerEvent(event.getContainerId(), AMContainerEventType.C_NM_STOP_SENT));
+ getContext().containerStopRequested(event.getContainerId());
}
- void launch(NMCommunicatorLaunchRequestEvent event) {
+ void launch(ContainerLaunchRequest event) {
// launch container by putting it in simulated container list
ContainerData cData = new ContainerData(event.getContainerId(),
event.getContainerLaunchContext());
containers.put(event.getContainerId(), cData);
containersToProcess.add(cData);
- getContext().getEventHandler().handle(new AMContainerEventLaunched(event.getContainerId()));
+ getContext().containerLaunched(event.getContainerId());
}
public void waitTillContainersLaunched() throws InterruptedException {
@@ -289,7 +283,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
}
void incrementTime(long inc) {
- Clock clock = getContext().getClock();
+ Clock clock = MockDAGAppMaster.this.getContext().getClock();
if (clock instanceof MockClock) {
((MockClock) clock).incrementTime(inc);
}
@@ -493,7 +487,8 @@ public class MockDAGAppMaster extends DAGAppMaster {
super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
isSession, workingDirectory, localDirs, logDirs, new TezApiVersionInfo().getVersion(), 1,
credentials, jobUserName);
- containerLauncher = new MockContainerLauncher(launcherGoFlag);
+ containerLauncherContext = new ContainerLauncherContextImpl(getContext(), getTaskAttemptListener());
+ containerLauncher = new MockContainerLauncher(launcherGoFlag, containerLauncherContext);
shutdownHandler = new MockDAGAppMasterShutdownHandler();
this.initFailFlag = initFailFlag;
this.startFailFlag = startFailFlag;
@@ -508,7 +503,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
String[] containerLaunchers,
boolean isLocal)
throws UnknownHostException {
- return new ContainerLauncherRouter(containerLauncher);
+ return new ContainerLauncherRouter(containerLauncher, getContext());
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 7f0362d..df643e4 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -51,8 +51,8 @@ import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.common.security.TokenCache;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezException;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
index 934543f..8d776fb 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 62edac9..e37ab4a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -53,12 +53,11 @@ import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.tez.common.MockDNSToSwitchMapping;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.OutputDescriptor;
import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
@@ -94,7 +93,6 @@ import org.apache.tez.runtime.api.impl.OutputSpec;
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.junit.BeforeClass;
import org.junit.Test;
-import org.mockito.internal.matchers.Null;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
index 0a642bb..b555c62 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index b8b4774..7bcb6d2 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -63,8 +63,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.TokenCache;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerHeartbeatHandler;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index 9d22196..dbf5054 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -14,34 +14,30 @@
package org.apache.tez.dag.app.launcher;
+import java.io.IOException;
import java.net.InetSocketAddress;
import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.util.Clock;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.app.TezTestServiceCommunicator;
-import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
-import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEventLaunchFailed;
-import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
-import org.apache.tez.dag.app.rm.container.AMContainerEventType;
-import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
import org.apache.tez.service.TezTestServiceConfConstants;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class TezTestServiceContainerLauncher extends AbstractService implements ContainerLauncher {
+// TODO TEZ-2003 look for all LOG.*(DBG and LOG.*(DEBUG messages
+
+public class TezTestServiceContainerLauncher extends ContainerLauncher {
// TODO Support interruptability of tasks which haven't yet been launched.
@@ -49,40 +45,32 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
static final Logger LOG = LoggerFactory.getLogger(TezTestServiceContainerLauncher.class);
- private final AppContext context;
private final String tokenIdentifier;
- private final TaskAttemptListener tal;
private final int servicePort;
private final TezTestServiceCommunicator communicator;
- private final Clock clock;
private final ApplicationAttemptId appAttemptId;
+ // private final TaskAttemptListener tal;
// Configuration passed in here to set up final parameters
- public TezTestServiceContainerLauncher(AppContext appContext, Configuration conf,
- TaskAttemptListener tal) {
- super(TezTestServiceContainerLauncher.class.getName());
- this.clock = appContext.getClock();
- int numThreads = conf.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS,
+ public TezTestServiceContainerLauncher(ContainerLauncherContext containerLauncherContext) {
+ super(TezTestServiceContainerLauncher.class.getName(), containerLauncherContext);
+ int numThreads = getContext().getInitialConfiguration().getInt(
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS,
TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT);
- this.servicePort = conf.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT, -1);
+ this.servicePort = getContext().getInitialConfiguration().getInt(
+ TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT, -1);
Preconditions.checkArgument(servicePort > 0,
TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT + " must be set");
this.communicator = new TezTestServiceCommunicator(numThreads);
- this.context = appContext;
- this.tokenIdentifier = context.getApplicationID().toString();
- this.appAttemptId = appContext.getApplicationAttemptId();
- this.tal = tal;
- }
-
- @Override
- public void serviceInit(Configuration conf) {
- communicator.init(conf);
+ this.tokenIdentifier = getContext().getApplicationAttemptId().getApplicationId().toString();
+ this.appAttemptId = getContext().getApplicationAttemptId();
}
@Override
public void serviceStart() {
+ communicator.init(getContext().getInitialConfiguration());
communicator.start();
}
@@ -92,51 +80,56 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
}
@Override
- public void handle(NMCommunicatorEvent event) {
- switch (event.getType()) {
- case CONTAINER_LAUNCH_REQUEST:
- final NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
- RunContainerRequestProto runRequest = constructRunContainerRequest(launchEvent);
- communicator.runContainer(runRequest, launchEvent.getNodeId().getHost(),
- launchEvent.getNodeId().getPort(),
- new TezTestServiceCommunicator.ExecuteRequestCallback<TezTestServiceProtocolProtos.RunContainerResponseProto>() {
- @Override
- public void setResponse(TezTestServiceProtocolProtos.RunContainerResponseProto response) {
- LOG.info("Container: " + launchEvent.getContainerId() + " launch succeeded on host: " + launchEvent.getNodeId());
- context.getEventHandler().handle(new AMContainerEventLaunched(launchEvent.getContainerId()));
- ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
- launchEvent.getContainerId(), clock.getTime(), context.getApplicationAttemptId());
- context.getHistoryHandler().handle(new DAGHistoryEvent(
- null, lEvt));
- }
-
- @Override
- public void indicateError(Throwable t) {
- LOG.error("Failed to launch container: " + launchEvent.getContainer() + " on host: " + launchEvent.getNodeId(), t);
- sendContainerLaunchFailedMsg(launchEvent.getContainerId(), t);
- }
- });
- break;
- case CONTAINER_STOP_REQUEST:
- LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + event);
- // that the container is actually done (normally received from RM)
- // TODO Sending this out for an un-launched container is invalid
- context.getEventHandler().handle(new AMContainerEvent(event.getContainerId(),
- AMContainerEventType.C_NM_STOP_SENT));
- break;
+ public void launchContainer(final ContainerLaunchRequest launchRequest) {
+ RunContainerRequestProto runRequest = null;
+ try {
+ runRequest = constructRunContainerRequest(launchRequest);
+ } catch (IOException e) {
+ getContext().containerLaunchFailed(launchRequest.getContainerId(),
+ "Failed to construct launch request, " + StringUtils.stringifyException(e));
+ return;
}
+ communicator.runContainer(runRequest, launchRequest.getNodeId().getHost(),
+ launchRequest.getNodeId().getPort(),
+ new TezTestServiceCommunicator.ExecuteRequestCallback<TezTestServiceProtocolProtos.RunContainerResponseProto>() {
+ @Override
+ public void setResponse(TezTestServiceProtocolProtos.RunContainerResponseProto response) {
+ LOG.info(
+ "Container: " + launchRequest.getContainerId() + " launch succeeded on host: " +
+ launchRequest.getNodeId());
+ getContext().containerLaunched(launchRequest.getContainerId());
+ }
+
+ @Override
+ public void indicateError(Throwable t) {
+ LOG.error(
+ "Failed to launch container: " + launchRequest.getContainerId() + " on host: " +
+ launchRequest.getNodeId(), t);
+ sendContainerLaunchFailedMsg(launchRequest.getContainerId(), t);
+ }
+ });
}
- private RunContainerRequestProto constructRunContainerRequest(NMCommunicatorLaunchRequestEvent event) {
+ @Override
+ public void stopContainer(ContainerStopRequest stopRequest) {
+ LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + stopRequest);
+ // that the container is actually done (normally received from RM)
+ // TODO Sending this out for an un-launched container is invalid
+ getContext().containerStopRequested(stopRequest.getContainerId());
+ }
+
+ private RunContainerRequestProto constructRunContainerRequest(ContainerLaunchRequest launchRequest) throws
+ IOException {
RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
- InetSocketAddress address = tal.getTaskCommunicator(event.getTaskCommId()).getAddress();
+ Preconditions.checkArgument(launchRequest.getTaskCommunicatorName().equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT));
+ InetSocketAddress address = (InetSocketAddress) getContext().getTaskCommunicatorMetaInfo(launchRequest.getTaskCommunicatorName());
builder.setAmHost(address.getHostName()).setAmPort(address.getPort());
builder.setAppAttemptNumber(appAttemptId.getAttemptId());
builder.setApplicationIdString(appAttemptId.getApplicationId().toString());
builder.setTokenIdentifier(tokenIdentifier);
- builder.setContainerIdString(event.getContainer().getId().toString());
+ builder.setContainerIdString(launchRequest.getContainerId().toString());
builder.setCredentialsBinary(
- ByteString.copyFrom(event.getContainerLaunchContext().getTokens()));
+ ByteString.copyFrom(launchRequest.getContainerLaunchContext().getTokens()));
// TODO Avoid reading this from the environment
builder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
return builder.build();
@@ -144,6 +137,8 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
@SuppressWarnings("unchecked")
void sendContainerLaunchFailedMsg(ContainerId containerId, Throwable t) {
- context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, t == null ? "" : t.getMessage()));
+ getContext().containerLaunchFailed(containerId, t == null ? "" : t.getMessage());
}
+
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
index 977d0d3..d3743e1 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
@@ -14,53 +14,32 @@
package org.apache.tez.dag.app.launcher;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.util.Clock;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.TaskAttemptListener;
-import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
-import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
-import org.apache.tez.dag.app.rm.container.AMContainerEventType;
-import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class TezTestServiceNoOpContainerLauncher extends AbstractService implements ContainerLauncher {
+public class TezTestServiceNoOpContainerLauncher extends ContainerLauncher {
static final Logger LOG = LoggerFactory.getLogger(TezTestServiceNoOpContainerLauncher.class);
- private final AppContext context;
- private final Clock clock;
- public TezTestServiceNoOpContainerLauncher(AppContext appContext, Configuration conf,
- TaskAttemptListener tal) {
- super(TezTestServiceNoOpContainerLauncher.class.getName());
- this.context = appContext;
- this.clock = appContext.getClock();
+ public TezTestServiceNoOpContainerLauncher(ContainerLauncherContext containerLauncherContext) {
+ super(TezTestServiceNoOpContainerLauncher.class.getName(), containerLauncherContext);
}
@Override
- public void handle(NMCommunicatorEvent event) {
- switch(event.getType()) {
- case CONTAINER_LAUNCH_REQUEST:
- final NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
- LOG.info("No-op launch for container: " + launchEvent.getContainerId() + " succeeded on host: " + launchEvent.getNodeId());
- context.getEventHandler().handle(new AMContainerEventLaunched(launchEvent.getContainerId()));
- ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
- launchEvent.getContainerId(), clock.getTime(), context.getApplicationAttemptId());
- context.getHistoryHandler().handle(new DAGHistoryEvent(
- null, lEvt));
- break;
- case CONTAINER_STOP_REQUEST:
- LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + event);
- context.getEventHandler().handle(new AMContainerEvent(event.getContainerId(),
- AMContainerEventType.C_NM_STOP_SENT));
- break;
- }
+ public void launchContainer(ContainerLaunchRequest launchRequest) {
+ LOG.info("No-op launch for container {} succeeded on host: {}", launchRequest.getContainerId(),
+ launchRequest.getNodeId());
+ getContext().containerLaunched(launchRequest.getContainerId());
+ }
+ @Override
+ public void stopContainer(ContainerStopRequest stopRequest) {
+ LOG.info("DEBUG: Ignoring STOP_REQUEST {}", stopRequest);
+ getContext().containerStopRequested(stopRequest.getContainerId());
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 073cb50..506e991 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.service.TezTestServiceConfConstants;
import org.slf4j.Logger;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index 98673a6..444498e 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
import org.apache.tez.dag.app.TezTestServiceCommunicator;
[47/51] [abbrv] tez git commit: TEZ-2657. Add tests for client side
changes - specifying plugins, etc. (sseth)
Posted by ss...@apache.org.
TEZ-2657. Add tests for client side changes - specifying plugins, etc. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/306020d2
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/306020d2
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/306020d2
Branch: refs/heads/TEZ-2003
Commit: 306020d20ad4c23bb9854e9e7fc029f2096948c6
Parents: eb82ca2
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Jul 29 18:26:01 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:58 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../java/org/apache/tez/client/TezClient.java | 15 +-
.../org/apache/tez/client/TezClientUtils.java | 38 +---
.../apache/tez/dag/api/DagTypeConverters.java | 67 +++++--
.../java/org/apache/tez/dag/api/Vertex.java | 41 ++++
.../api/ServicePluginsDescriptor.java | 36 ++++
tez-api/src/main/proto/DAGApiRecords.proto | 2 +-
.../org/apache/tez/client/TestTezClient.java | 113 +++++++++--
.../apache/tez/client/TestTezClientUtils.java | 16 +-
.../org/apache/tez/dag/api/TestDAGPlan.java | 63 +++++-
.../tez/dag/api/TestDagTypeConverters.java | 196 ++++++++++++++++++-
.../org/apache/tez/dag/app/DAGAppMaster.java | 4 +-
12 files changed, 508 insertions(+), 84 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9d72d92..9b3967a 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -40,5 +40,6 @@ ALL CHANGES:
TEZ-2652. Cleanup the way services are specified for an AM and vertices.
TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
TEZ-2441. Add tests for TezTaskRunner2.
+ TEZ-2657. Add tests for client side changes - specifying plugins, etc.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index f961291..7778ef3 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -94,13 +94,16 @@ public class TezClient {
@VisibleForTesting
static final String NO_CLUSTER_DIAGNOSTICS_MSG = "No cluster diagnostics found.";
- private final String clientName;
+ @VisibleForTesting
+ final String clientName;
private ApplicationId sessionAppId;
private ApplicationId lastSubmittedAppId;
- private AMConfiguration amConfig;
+ @VisibleForTesting
+ final AMConfiguration amConfig;
private FrameworkClient frameworkClient;
private String diagnostics;
- private boolean isSession;
+ @VisibleForTesting
+ final boolean isSession;
private boolean sessionStarted = false;
private boolean sessionStopped = false;
/** Tokens which will be required for all DAGs submitted to this session. */
@@ -112,8 +115,10 @@ public class TezClient {
private JobTokenSecretManager jobTokenSecretManager =
new JobTokenSecretManager();
private final Map<String, LocalResource> additionalLocalResources = Maps.newHashMap();
- private final TezApiVersionInfo apiVersionInfo;
- private final ServicePluginsDescriptor servicePluginsDescriptor;
+ @VisibleForTesting
+ final TezApiVersionInfo apiVersionInfo;
+ @VisibleForTesting
+ final ServicePluginsDescriptor servicePluginsDescriptor;
private HistoryACLPolicyManager historyACLPolicyManager;
private int preWarmDAGCounter = 0;
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index 9cf1f3f..6086fa1 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -39,9 +39,7 @@ import java.util.Map.Entry;
import com.google.common.base.Strings;
import org.apache.commons.lang.StringUtils;
-import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
-import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -778,47 +776,13 @@ public class TezClientUtils {
}
AMPluginDescriptorProto pluginDescriptorProto =
- createAMServicePluginDescriptorProto(servicePluginsDescriptor);
+ DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
builder.setAmPluginDescriptor(pluginDescriptorProto);
return builder.build();
}
- static AMPluginDescriptorProto createAMServicePluginDescriptorProto(
- ServicePluginsDescriptor servicePluginsDescriptor) {
- AMPluginDescriptorProto.Builder pluginDescriptorBuilder =
- AMPluginDescriptorProto.newBuilder();
- if (servicePluginsDescriptor != null) {
- pluginDescriptorBuilder.setContainersEnabled(servicePluginsDescriptor.areContainersEnabled());
- pluginDescriptorBuilder.setUberEnabled(servicePluginsDescriptor.isUberEnabled());
-
- if (servicePluginsDescriptor.getTaskSchedulerDescriptors() != null &&
- servicePluginsDescriptor.getTaskSchedulerDescriptors().length > 0) {
- List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
- servicePluginsDescriptor.getTaskSchedulerDescriptors());
- pluginDescriptorBuilder.addAllTaskScedulers(namedEntityProtos);
- }
-
- if (servicePluginsDescriptor.getContainerLauncherDescriptors() != null &&
- servicePluginsDescriptor.getContainerLauncherDescriptors().length > 0) {
- List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
- servicePluginsDescriptor.getContainerLauncherDescriptors());
- pluginDescriptorBuilder.addAllContainerLaunchers(namedEntityProtos);
- }
-
- if (servicePluginsDescriptor.getTaskCommunicatorDescriptors() != null &&
- servicePluginsDescriptor.getTaskCommunicatorDescriptors().length > 0) {
- List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
- servicePluginsDescriptor.getTaskCommunicatorDescriptors());
- pluginDescriptorBuilder.addAllTaskCommunicators(namedEntityProtos);
- }
-
- } else {
- pluginDescriptorBuilder.setContainersEnabled(true).setUberEnabled(false);
- }
- return pluginDescriptorBuilder.build();
- }
/**
* Helper function to create a YARN LocalResource
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
index 2e0d417..61e4d33 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
@@ -52,9 +52,11 @@ import org.apache.tez.common.counters.TezCounters;
import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
import org.apache.tez.dag.api.client.StatusGetOpts;
import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto;
import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataMovementType;
@@ -74,14 +76,13 @@ import org.apache.tez.dag.api.records.DAGProtos.TezCounterProto;
import org.apache.tez.dag.api.records.DAGProtos.TezCountersProto;
import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto;
import org.apache.tez.dag.api.records.DAGProtos.VertexLocationHintProto;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
import com.google.protobuf.ByteString.Output;
-import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
-import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
-import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
@Private
public class DagTypeConverters {
@@ -732,13 +733,13 @@ public class DagTypeConverters {
return payload.getPayload();
}
- public static DAGProtos.VertexExecutionContextProto convertToProto(
- Vertex.VertexExecutionContext context) {
+ public static VertexExecutionContextProto convertToProto(
+ VertexExecutionContext context) {
if (context == null) {
return null;
} else {
- DAGProtos.VertexExecutionContextProto.Builder builder =
- DAGProtos.VertexExecutionContextProto.newBuilder();
+ VertexExecutionContextProto.Builder builder =
+ VertexExecutionContextProto.newBuilder();
builder.setExecuteInAm(context.shouldExecuteInAm());
builder.setExecuteInContainers(context.shouldExecuteInContainers());
if (context.getTaskSchedulerName() != null) {
@@ -754,26 +755,26 @@ public class DagTypeConverters {
}
}
- public static Vertex.VertexExecutionContext convertFromProto(
- DAGProtos.VertexExecutionContextProto proto) {
+ public static VertexExecutionContext convertFromProto(
+ VertexExecutionContextProto proto) {
if (proto == null) {
return null;
} else {
if (proto.getExecuteInAm()) {
- Vertex.VertexExecutionContext context =
- Vertex.VertexExecutionContext.createExecuteInAm(proto.getExecuteInAm());
+ VertexExecutionContext context =
+ VertexExecutionContext.createExecuteInAm(proto.getExecuteInAm());
return context;
} else if (proto.getExecuteInContainers()) {
- Vertex.VertexExecutionContext context =
- Vertex.VertexExecutionContext.createExecuteInContainers(proto.getExecuteInContainers());
+ VertexExecutionContext context =
+ VertexExecutionContext.createExecuteInContainers(proto.getExecuteInContainers());
return context;
} else {
String taskScheduler = proto.hasTaskSchedulerName() ? proto.getTaskSchedulerName() : null;
String containerLauncher =
proto.hasContainerLauncherName() ? proto.getContainerLauncherName() : null;
String taskComm = proto.hasTaskCommName() ? proto.getTaskCommName() : null;
- Vertex.VertexExecutionContext context =
- Vertex.VertexExecutionContext.create(taskScheduler, containerLauncher, taskComm);
+ VertexExecutionContext context =
+ VertexExecutionContext.create(taskScheduler, containerLauncher, taskComm);
return context;
}
}
@@ -800,4 +801,40 @@ public class DagTypeConverters {
return builder.build();
}
+ public static AMPluginDescriptorProto convertServicePluginDescriptoToProto(
+ ServicePluginsDescriptor servicePluginsDescriptor) {
+ AMPluginDescriptorProto.Builder pluginDescriptorBuilder =
+ AMPluginDescriptorProto.newBuilder();
+ if (servicePluginsDescriptor != null) {
+
+ pluginDescriptorBuilder.setContainersEnabled(servicePluginsDescriptor.areContainersEnabled());
+ pluginDescriptorBuilder.setUberEnabled(servicePluginsDescriptor.isUberEnabled());
+
+ if (servicePluginsDescriptor.getTaskSchedulerDescriptors() != null &&
+ servicePluginsDescriptor.getTaskSchedulerDescriptors().length > 0) {
+ List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+ servicePluginsDescriptor.getTaskSchedulerDescriptors());
+ pluginDescriptorBuilder.addAllTaskSchedulers(namedEntityProtos);
+ }
+
+ if (servicePluginsDescriptor.getContainerLauncherDescriptors() != null &&
+ servicePluginsDescriptor.getContainerLauncherDescriptors().length > 0) {
+ List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+ servicePluginsDescriptor.getContainerLauncherDescriptors());
+ pluginDescriptorBuilder.addAllContainerLaunchers(namedEntityProtos);
+ }
+
+ if (servicePluginsDescriptor.getTaskCommunicatorDescriptors() != null &&
+ servicePluginsDescriptor.getTaskCommunicatorDescriptors().length > 0) {
+ List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+ servicePluginsDescriptor.getTaskCommunicatorDescriptors());
+ pluginDescriptorBuilder.addAllTaskCommunicators(namedEntityProtos);
+ }
+
+ } else {
+ pluginDescriptorBuilder.setContainersEnabled(true).setUberEnabled(false);
+ }
+ return pluginDescriptorBuilder.build();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
index 34124b2..8953ae1 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
@@ -511,6 +511,47 @@ public class Vertex {
", taskCommName='" + taskCommName + '\'' +
'}';
}
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ VertexExecutionContext that = (VertexExecutionContext) o;
+
+ if (executeInAm != that.executeInAm) {
+ return false;
+ }
+ if (executeInContainers != that.executeInContainers) {
+ return false;
+ }
+ if (taskSchedulerName != null ? !taskSchedulerName.equals(that.taskSchedulerName) :
+ that.taskSchedulerName != null) {
+ return false;
+ }
+ if (containerLauncherName != null ?
+ !containerLauncherName.equals(that.containerLauncherName) :
+ that.containerLauncherName != null) {
+ return false;
+ }
+ return !(taskCommName != null ? !taskCommName.equals(that.taskCommName) :
+ that.taskCommName != null);
+
+ }
+
+ @Override
+ public int hashCode() {
+ int result = (executeInAm ? 1 : 0);
+ result = 31 * result + (executeInContainers ? 1 : 0);
+ result = 31 * result + (taskSchedulerName != null ? taskSchedulerName.hashCode() : 0);
+ result = 31 * result + (containerLauncherName != null ? containerLauncherName.hashCode() : 0);
+ result = 31 * result + (taskCommName != null ? taskCommName.hashCode() : 0);
+ return result;
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
index 8df102a..2e4fc46 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
@@ -46,6 +46,15 @@ public class ServicePluginsDescriptor {
this.taskCommunicatorDescriptors = taskCommunicatorDescriptors;
}
+ /**
+ * Create a service plugin descriptor with the provided plugins. Regular containers will also be enabled
+ * when using this method.
+ *
+ * @param taskSchedulerDescriptor the task scheduler plugin descriptors
+ * @param containerLauncherDescriptors the container launcher plugin descriptors
+ * @param taskCommunicatorDescriptors the task communicator plugin descriptors
+ * @return
+ */
public static ServicePluginsDescriptor create(TaskSchedulerDescriptor[] taskSchedulerDescriptor,
ContainerLauncherDescriptor[] containerLauncherDescriptors,
TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
@@ -53,6 +62,15 @@ public class ServicePluginsDescriptor {
containerLauncherDescriptors, taskCommunicatorDescriptors);
}
+ /**
+ * Create a service plugin descriptor with the provided plugins. Also allows specification of whether
+ * in-AM execution is enabled. Container execution is enabled by default.
+ * @param enableUber whether to enable execution in the AM or not
+ * @param taskSchedulerDescriptor the task scheduler plugin descriptors
+ * @param containerLauncherDescriptors the container launcher plugin descriptors
+ * @param taskCommunicatorDescriptors the task communicator plugin descriptors
+ * @return
+ */
public static ServicePluginsDescriptor create(boolean enableUber,
TaskSchedulerDescriptor[] taskSchedulerDescriptor,
ContainerLauncherDescriptor[] containerLauncherDescriptors,
@@ -61,6 +79,17 @@ public class ServicePluginsDescriptor {
containerLauncherDescriptors, taskCommunicatorDescriptors);
}
+ /**
+ * Create a service plugin descriptor with the provided plugins. Also allows specification of whether
+ * container execution and in-AM execution will be enabled.
+ *
+ * @param enableContainers whether to enable execution in containers
+ * @param enableUber whether to enable execution in the AM or not
+ * @param taskSchedulerDescriptor the task scheduler plugin descriptors
+ * @param containerLauncherDescriptors the container launcher plugin descriptors
+ * @param taskCommunicatorDescriptors the task communicator plugin descriptors
+ * @return
+ */
public static ServicePluginsDescriptor create(boolean enableContainers, boolean enableUber,
TaskSchedulerDescriptor[] taskSchedulerDescriptor,
ContainerLauncherDescriptor[] containerLauncherDescriptors,
@@ -69,6 +98,13 @@ public class ServicePluginsDescriptor {
containerLauncherDescriptors, taskCommunicatorDescriptors);
}
+ /**
+ * Create a service plugin descriptor which may have in-AM execution of tasks enabled. Container
+ * execution is enabled by default
+ *
+ * @param enableUber whether to enable execution in the AM or not
+ * @return
+ */
public static ServicePluginsDescriptor create(boolean enableUber) {
return new ServicePluginsDescriptor(true, enableUber, null, null, null);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/main/proto/DAGApiRecords.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto
index ebe3259..193f7b8 100644
--- a/tez-api/src/main/proto/DAGApiRecords.proto
+++ b/tez-api/src/main/proto/DAGApiRecords.proto
@@ -180,7 +180,7 @@ message TezNamedEntityDescriptorProto {
message AMPluginDescriptorProto {
optional bool containers_enabled = 1 [default = true];
optional bool uber_enabled = 2 [default = false];
- repeated TezNamedEntityDescriptorProto task_scedulers = 3;
+ repeated TezNamedEntityDescriptorProto task_schedulers = 3;
repeated TezNamedEntityDescriptorProto container_launchers = 4;
repeated TezNamedEntityDescriptorProto task_communicators = 5;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
index dc0fbb1..7a642e6 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
@@ -26,6 +26,11 @@ import java.util.concurrent.atomic.AtomicReference;
import javax.annotation.Nullable;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
@@ -61,6 +66,7 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRespo
import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto;
import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto;
import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
import org.hamcrest.CoreMatchers;
import org.junit.Assert;
import org.junit.Test;
@@ -153,11 +159,11 @@ public class TestTezClient {
verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
ApplicationSubmissionContext context = captor.getValue();
Assert.assertEquals(3, context.getAMContainerSpec().getLocalResources().size());
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
TezConstants.TEZ_PB_BINARY_CONF_NAME));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
lrName1));
} else {
verify(client.mockYarnClient, times(0)).submitApplication(captor.capture());
@@ -172,7 +178,7 @@ public class TestTezClient {
DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG);
DAGClient dagClient = client.submitDAG(dag);
- Assert.assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
+ assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
if (isSession) {
verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
@@ -181,13 +187,13 @@ public class TestTezClient {
verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
ApplicationSubmissionContext context = captor.getValue();
Assert.assertEquals(4, context.getAMContainerSpec().getLocalResources().size());
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
TezConstants.TEZ_PB_BINARY_CONF_NAME));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
TezConstants.TEZ_PB_PLAN_BINARY_NAME));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
lrName1));
}
@@ -211,7 +217,7 @@ public class TestTezClient {
if (isSession) {
// same app master
verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
- Assert.assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
+ assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
// additional resource is sent
ArgumentCaptor<SubmitDAGRequestProto> captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class);
verify(client.sessionAmProxy, times(2)).submitDAG((RpcController)any(), captor1.capture());
@@ -220,20 +226,20 @@ public class TestTezClient {
Assert.assertEquals(lrName2, proto.getAdditionalAmResources().getLocalResources(0).getName());
} else {
// new app master
- Assert.assertTrue(dagClient.getExecutionContext().contains(appId2.toString()));
+ assertTrue(dagClient.getExecutionContext().contains(appId2.toString()));
verify(client.mockYarnClient, times(2)).submitApplication(captor.capture());
// additional resource is added
ApplicationSubmissionContext context = captor.getValue();
Assert.assertEquals(5, context.getAMContainerSpec().getLocalResources().size());
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
TezConstants.TEZ_PB_BINARY_CONF_NAME));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
TezConstants.TEZ_PB_PLAN_BINARY_NAME));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
lrName1));
- Assert.assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
+ assertTrue(context.getAMContainerSpec().getLocalResources().containsKey(
lrName2));
}
@@ -263,7 +269,7 @@ public class TestTezClient {
ArgumentCaptor<SubmitDAGRequestProto> captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class);
verify(client.sessionAmProxy, times(1)).submitDAG((RpcController)any(), captor1.capture());
SubmitDAGRequestProto proto = captor1.getValue();
- Assert.assertTrue(proto.getDAGPlan().getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX));
+ assertTrue(proto.getDAGPlan().getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX));
client.stop();
}
@@ -330,7 +336,7 @@ public class TestTezClient {
thread.join(250);
thread.interrupt();
thread.join();
- Assert.assertThat(exceptionReference.get(),CoreMatchers. instanceOf(InterruptedException.class));
+ Assert.assertThat(exceptionReference.get(), CoreMatchers.instanceOf(InterruptedException.class));
client.stop();
}
@@ -347,7 +353,7 @@ public class TestTezClient {
client.waitTillReady();
Assert.fail();
} catch (SessionNotRunning e) {
- Assert.assertTrue(e.getMessage().contains(msg));
+ assertTrue(e.getMessage().contains(msg));
}
client.stop();
}
@@ -362,7 +368,7 @@ public class TestTezClient {
client.waitTillReady();
Assert.fail();
} catch (SessionNotRunning e) {
- Assert.assertTrue(e.getMessage().contains(TezClient.NO_CLUSTER_DIAGNOSTICS_MSG));
+ assertTrue(e.getMessage().contains(TezClient.NO_CLUSTER_DIAGNOSTICS_MSG));
}
client.stop();
}
@@ -387,9 +393,76 @@ public class TestTezClient {
client.submitDAG(dag);
Assert.fail();
} catch (SessionNotRunning e) {
- Assert.assertTrue(e.getMessage().contains(msg));
+ assertTrue(e.getMessage().contains(msg));
}
client.stop();
}
+ @Test(timeout = 5000)
+ public void testClientBuilder() {
+ TezConfiguration tezConfWitSession = new TezConfiguration();
+ tezConfWitSession.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
+
+ TezConfiguration tezConfNoSession = new TezConfiguration();
+ tezConfNoSession.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false);
+
+ AMConfiguration amConf;
+ TezClient tezClient;
+ Credentials credentials = new Credentials();
+ Map<String, LocalResource> localResourceMap = new HashMap<>();
+ localResourceMap.put("testResource", mock(LocalResource.class));
+ ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor.create(true);
+
+ // Session mode via conf
+ tezClient = TezClient.newBuilder("client", tezConfWitSession).build();
+ assertTrue(tezClient.isSession);
+ assertNull(tezClient.servicePluginsDescriptor);
+ assertNotNull(tezClient.apiVersionInfo);
+ amConf = tezClient.amConfig;
+ assertNotNull(amConf);
+ assertEquals(0, amConf.getAMLocalResources().size());
+ assertNull(amConf.getCredentials());
+ assertTrue(
+ amConf.getTezConfiguration().getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false));
+
+ // Non-Session mode via conf
+ tezClient = TezClient.newBuilder("client", tezConfNoSession).build();
+ assertFalse(tezClient.isSession);
+ assertNull(tezClient.servicePluginsDescriptor);
+ assertNotNull(tezClient.apiVersionInfo);
+ amConf = tezClient.amConfig;
+ assertNotNull(amConf);
+ assertEquals(0, amConf.getAMLocalResources().size());
+ assertNull(amConf.getCredentials());
+ assertFalse(
+ amConf.getTezConfiguration().getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true));
+
+ // no-session via config. API explicit session.
+ tezClient = TezClient.newBuilder("client", tezConfNoSession).setIsSession(true).build();
+ assertTrue(tezClient.isSession);
+ assertNull(tezClient.servicePluginsDescriptor);
+ assertNotNull(tezClient.apiVersionInfo);
+ amConf = tezClient.amConfig;
+ assertNotNull(amConf);
+ assertEquals(0, amConf.getAMLocalResources().size());
+ assertNull(amConf.getCredentials());
+ assertTrue(
+ amConf.getTezConfiguration().getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false));
+
+ // Plugins, credentials, local resources
+ tezClient = TezClient.newBuilder("client", tezConfWitSession).setCredentials(credentials)
+ .setLocalResources(localResourceMap).setServicePluginDescriptor(servicePluginsDescriptor)
+ .build();
+ assertTrue(tezClient.isSession);
+ assertEquals(servicePluginsDescriptor, tezClient.servicePluginsDescriptor);
+ assertNotNull(tezClient.apiVersionInfo);
+ amConf = tezClient.amConfig;
+ assertNotNull(amConf);
+ assertEquals(1, amConf.getAMLocalResources().size());
+ assertEquals(localResourceMap, amConf.getAMLocalResources());
+ assertEquals(credentials, amConf.getCredentials());
+ assertTrue(
+ amConf.getTezConfiguration().getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false));
+ }
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
index 8946ef0..8f40bbd 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
@@ -70,6 +70,7 @@ import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.Vertex;
import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
import org.junit.Assert;
import org.junit.Test;
/**
@@ -500,7 +501,8 @@ public class TestTezClientUtils {
Assert.assertNotNull(javaOpts);
Assert.assertTrue(javaOpts.contains("-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=FOOBAR")
&& javaOpts.contains(TezConstants.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE)
- && javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator"));
+ &&
+ javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator"));
}
@Test (timeout = 5000)
@@ -677,6 +679,16 @@ public class TestTezClientUtils {
Assert.assertTrue(resourceNames.contains("dir2-f.txt"));
}
- // TODO TEZ-2003 Add test to validate ServicePluginDescriptor propagation
+ @Test(timeout = 5000)
+ public void testServiceDescriptorSerializationForAM() {
+ Configuration conf = new Configuration(false);
+ ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor.create(true);
+
+ ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(conf, null,
+ servicePluginsDescriptor);
+
+ assertTrue(confProto.hasAmPluginDescriptor());
+ assertTrue(confProto.getAmPluginDescriptor().getUberEnabled());
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
index fccbb08..cd42109 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
@@ -37,11 +37,14 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
+import org.apache.tez.dag.api.records.DAGProtos;
import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
+import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto;
import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
import org.junit.Assert;
import org.junit.Rule;
@@ -131,7 +134,8 @@ public class TestDAGPlan {
EdgeManagerPluginDescriptor emDesc = edgeProperty.getEdgeManagerDescriptor();
Assert.assertNotNull(emDesc);
Assert.assertEquals("emClass", emDesc.getClassName());
- Assert.assertTrue(Arrays.equals("emPayload".getBytes(), emDesc.getUserPayload().deepCopyAsArray()));
+ Assert.assertTrue(
+ Arrays.equals("emPayload".getBytes(), emDesc.getUserPayload().deepCopyAsArray()));
}
@Test(timeout = 5000)
@@ -311,4 +315,61 @@ public class TestDAGPlan {
assertNotNull(fetchedCredentials.getToken(new Text("Token1")));
assertNotNull(fetchedCredentials.getToken(new Text("Token2")));
}
+
+ @Test(timeout = 5000)
+ public void testServiceDescriptorPropagation() {
+ DAG dag = DAG.create("testDag");
+ ProcessorDescriptor pd1 = ProcessorDescriptor.create("processor1").
+ setUserPayload(UserPayload.create(ByteBuffer.wrap("processor1Bytes".getBytes())));
+ ProcessorDescriptor pd2 = ProcessorDescriptor.create("processor2").
+ setUserPayload(UserPayload.create(ByteBuffer.wrap("processor2Bytes".getBytes())));
+
+ VertexExecutionContext defaultExecutionContext =
+ VertexExecutionContext.create("plugin", "plugin", "plugin");
+ VertexExecutionContext v1Context = VertexExecutionContext.createExecuteInAm(true);
+
+
+ Vertex v1 = Vertex.create("v1", pd1, 10, Resource.newInstance(1024, 1)).setExecutionContext(v1Context);
+ Vertex v2 = Vertex.create("v2", pd2, 1, Resource.newInstance(1024, 1));
+ v1.setTaskLaunchCmdOpts("").setTaskEnvironment(new HashMap<String, String>())
+ .addTaskLocalFiles(new HashMap<String, LocalResource>());
+ v2.setTaskLaunchCmdOpts("").setTaskEnvironment(new HashMap<String, String>())
+ .addTaskLocalFiles(new HashMap<String, LocalResource>());
+
+ InputDescriptor inputDescriptor = InputDescriptor.create("input").
+ setUserPayload(UserPayload.create(ByteBuffer.wrap("inputBytes".getBytes())));
+ OutputDescriptor outputDescriptor = OutputDescriptor.create("output").
+ setUserPayload(UserPayload.create(ByteBuffer.wrap("outputBytes".getBytes())));
+ Edge edge = Edge.create(v1, v2, EdgeProperty.create(
+ DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+ SchedulingType.SEQUENTIAL, outputDescriptor, inputDescriptor));
+
+ dag.addVertex(v1).addVertex(v2).addEdge(edge);
+ dag.setExecutionContext(defaultExecutionContext);
+
+ DAGPlan dagProto = dag.createDag(new TezConfiguration(), null, null, null, true);
+
+ assertEquals(2, dagProto.getVertexCount());
+ assertEquals(1, dagProto.getEdgeCount());
+
+ assertTrue(dagProto.hasDefaultExecutionContext());
+ VertexExecutionContextProto defaultContextProto = dagProto.getDefaultExecutionContext();
+ assertFalse(defaultContextProto.getExecuteInContainers());
+ assertFalse(defaultContextProto.getExecuteInAm());
+ assertEquals("plugin", defaultContextProto.getTaskSchedulerName());
+ assertEquals("plugin", defaultContextProto.getContainerLauncherName());
+ assertEquals("plugin", defaultContextProto.getTaskCommName());
+
+ VertexPlan v1Proto = dagProto.getVertex(0);
+ assertTrue(v1Proto.hasExecutionContext());
+ VertexExecutionContextProto v1ContextProto = v1Proto.getExecutionContext();
+ assertFalse(v1ContextProto.getExecuteInContainers());
+ assertTrue(v1ContextProto.getExecuteInAm());
+ assertFalse(v1ContextProto.hasTaskSchedulerName());
+ assertFalse(v1ContextProto.hasContainerLauncherName());
+ assertFalse(v1ContextProto.hasTaskCommName());
+
+ VertexPlan v2Proto = dagProto.getVertex(1);
+ assertFalse(v2Proto.hasExecutionContext());
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java
index 51b179a..e37f849 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java
@@ -18,15 +18,32 @@
package org.apache.tez.dag.api;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.List;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.tez.common.TezCommonUtils;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
+import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
+import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
import org.junit.Assert;
import org.junit.Test;
@@ -43,7 +60,7 @@ public class TestDagTypeConverters {
DagTypeConverters.convertToDAGPlan(entityDescriptor);
Assert.assertEquals(payload.getVersion(), proto.getTezUserPayload().getVersion());
Assert.assertArrayEquals(payload.deepCopyAsArray(), proto.getTezUserPayload().getUserPayload().toByteArray());
- Assert.assertTrue(proto.hasHistoryText());
+ assertTrue(proto.hasHistoryText());
Assert.assertNotEquals(historytext, proto.getHistoryText());
Assert.assertEquals(historytext, new String(
TezCommonUtils.decompressByteStringToByteArray(proto.getHistoryText())));
@@ -89,4 +106,181 @@ public class TestDagTypeConverters {
Assert.assertEquals(2311, lr2UrlDeserialized.getPort());
}
+
+ @Test(timeout = 5000)
+ public void testVertexExecutionContextTranslation() {
+ VertexExecutionContext originalContext;
+ VertexExecutionContextProto contextProto;
+ VertexExecutionContext retrievedContext;
+
+
+ // Uber
+ originalContext = VertexExecutionContext.createExecuteInAm(true);
+ contextProto = DagTypeConverters.convertToProto(originalContext);
+ retrievedContext = DagTypeConverters.convertFromProto(contextProto);
+ assertEquals(originalContext, retrievedContext);
+
+ // Regular containers
+ originalContext = VertexExecutionContext.createExecuteInContainers(true);
+ contextProto = DagTypeConverters.convertToProto(originalContext);
+ retrievedContext = DagTypeConverters.convertFromProto(contextProto);
+ assertEquals(originalContext, retrievedContext);
+
+ // Custom
+ originalContext = VertexExecutionContext.create("plugin", "plugin", "plugin");
+ contextProto = DagTypeConverters.convertToProto(originalContext);
+ retrievedContext = DagTypeConverters.convertFromProto(contextProto);
+ assertEquals(originalContext, retrievedContext);
+ }
+
+
+ static final String testScheduler = "testScheduler";
+ static final String testLauncher = "testLauncher";
+ static final String testComm = "testComm";
+ static final String classSuffix = "_class";
+
+ @Test(timeout = 5000)
+ public void testServiceDescriptorTranslation() {
+
+
+ TaskSchedulerDescriptor[] taskSchedulers;
+ ContainerLauncherDescriptor[] containerLaunchers;
+ TaskCommunicatorDescriptor[] taskComms;
+
+ ServicePluginsDescriptor servicePluginsDescriptor;
+ AMPluginDescriptorProto proto;
+
+ // Uber-execution
+ servicePluginsDescriptor = ServicePluginsDescriptor.create(true);
+ proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+ assertTrue(proto.hasUberEnabled());
+ assertTrue(proto.hasContainersEnabled());
+ assertTrue(proto.getUberEnabled());
+ assertTrue(proto.getContainersEnabled());
+ assertEquals(0, proto.getTaskSchedulersCount());
+ assertEquals(0, proto.getContainerLaunchersCount());
+ assertEquals(0, proto.getTaskCommunicatorsCount());
+
+ // Single plugin set specified. One with a payload.
+ taskSchedulers = createTaskScheduelrs(1, false);
+ containerLaunchers = createContainerLaunchers(1, false);
+ taskComms = createTaskCommunicators(1, true);
+
+ servicePluginsDescriptor = ServicePluginsDescriptor.create(taskSchedulers, containerLaunchers,
+ taskComms);
+ proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+ assertTrue(proto.hasUberEnabled());
+ assertTrue(proto.hasContainersEnabled());
+ assertFalse(proto.getUberEnabled());
+ assertTrue(proto.getContainersEnabled());
+ verifyPlugins(proto.getTaskSchedulersList(), 1, testScheduler, false);
+ verifyPlugins(proto.getContainerLaunchersList(), 1, testLauncher, false);
+ verifyPlugins(proto.getTaskCommunicatorsList(), 1, testComm, true);
+
+
+ // Multiple plugin set specified. All with a payload
+ taskSchedulers = createTaskScheduelrs(3, true);
+ containerLaunchers = createContainerLaunchers(3, true);
+ taskComms = createTaskCommunicators(3, true);
+
+ servicePluginsDescriptor = ServicePluginsDescriptor.create(taskSchedulers, containerLaunchers,
+ taskComms);
+ proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+ assertTrue(proto.hasUberEnabled());
+ assertTrue(proto.hasContainersEnabled());
+ assertFalse(proto.getUberEnabled());
+ assertTrue(proto.getContainersEnabled());
+ verifyPlugins(proto.getTaskSchedulersList(), 3, testScheduler, true);
+ verifyPlugins(proto.getContainerLaunchersList(), 3, testLauncher, true);
+ verifyPlugins(proto.getTaskCommunicatorsList(), 3, testComm, true);
+
+ // Single plugin set specified. One with a payload. No container execution. Uber enabled.
+ taskSchedulers = createTaskScheduelrs(1, false);
+ containerLaunchers = createContainerLaunchers(1, false);
+ taskComms = createTaskCommunicators(1, true);
+
+ servicePluginsDescriptor = ServicePluginsDescriptor.create(false, true, taskSchedulers, containerLaunchers,
+ taskComms);
+ proto = DagTypeConverters.convertServicePluginDescriptoToProto(servicePluginsDescriptor);
+ assertTrue(proto.hasUberEnabled());
+ assertTrue(proto.hasContainersEnabled());
+ assertTrue(proto.getUberEnabled());
+ assertFalse(proto.getContainersEnabled());
+ verifyPlugins(proto.getTaskSchedulersList(), 1, testScheduler, false);
+ verifyPlugins(proto.getContainerLaunchersList(), 1, testLauncher, false);
+ verifyPlugins(proto.getTaskCommunicatorsList(), 1, testComm, true);
+ }
+
+ private void verifyPlugins(List<TezNamedEntityDescriptorProto> entities, int expectedCount,
+ String baseString, boolean hasPayload) {
+ assertEquals(expectedCount, entities.size());
+ for (int i = 0; i < expectedCount; i++) {
+ assertEquals(indexedEntity(baseString, i), entities.get(i).getName());
+ TezEntityDescriptorProto subEntityProto = entities.get(i).getEntityDescriptor();
+ assertEquals(append(indexedEntity(baseString, i), classSuffix),
+ subEntityProto.getClassName());
+ assertEquals(hasPayload, subEntityProto.hasTezUserPayload());
+ if (hasPayload) {
+ UserPayload userPayload =
+ UserPayload
+ .create(subEntityProto.getTezUserPayload().getUserPayload().asReadOnlyByteBuffer(),
+ subEntityProto.getTezUserPayload().getVersion());
+ ByteBuffer bb = userPayload.getPayload();
+ assertNotNull(bb);
+ assertEquals(i, bb.getInt());
+ }
+ }
+ }
+
+ private TaskSchedulerDescriptor[] createTaskScheduelrs(int count, boolean withUserPayload) {
+ TaskSchedulerDescriptor[] descriptors = new TaskSchedulerDescriptor[count];
+ for (int i = 0; i < count; i++) {
+ descriptors[i] = TaskSchedulerDescriptor.create(indexedEntity(testScheduler, i),
+ append(indexedEntity(testScheduler, i), classSuffix));
+ if (withUserPayload) {
+ descriptors[i].setUserPayload(createPayload(i));
+ }
+ }
+ return descriptors;
+ }
+
+ private ContainerLauncherDescriptor[] createContainerLaunchers(int count,
+ boolean withUserPayload) {
+ ContainerLauncherDescriptor[] descriptors = new ContainerLauncherDescriptor[count];
+ for (int i = 0; i < count; i++) {
+ descriptors[i] = ContainerLauncherDescriptor.create(indexedEntity(testLauncher, i),
+ append(indexedEntity(testLauncher, i), classSuffix));
+ if (withUserPayload) {
+ descriptors[i].setUserPayload(createPayload(i));
+ }
+ }
+ return descriptors;
+ }
+
+ private TaskCommunicatorDescriptor[] createTaskCommunicators(int count, boolean withUserPayload) {
+ TaskCommunicatorDescriptor[] descriptors = new TaskCommunicatorDescriptor[count];
+ for (int i = 0; i < count; i++) {
+ descriptors[i] = TaskCommunicatorDescriptor.create(indexedEntity(testComm, i),
+ append(indexedEntity(testComm, i), classSuffix));
+ if (withUserPayload) {
+ descriptors[i].setUserPayload(createPayload(i));
+ }
+ }
+ return descriptors;
+ }
+
+ private static UserPayload createPayload(int i) {
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, i);
+ UserPayload payload = UserPayload.create(bb);
+ return payload;
+ }
+
+ private String indexedEntity(String name, int index) {
+ return name + index;
+ }
+
+ private String append(String s1, String s2) {
+ return s1 + s2;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/306020d2/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 4128841..9b16a90 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -409,9 +409,9 @@ public class DAGAppMaster extends AbstractService {
}
taskSchedulerDescriptors = parsePlugin(taskSchedulers,
- (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskScedulersCount() == 0 ?
+ (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskSchedulersCount() == 0 ?
null :
- amPluginDescriptorProto.getTaskScedulersList()),
+ amPluginDescriptorProto.getTaskSchedulersList()),
tezYarnEnabled, uberEnabled);
containerLauncherDescriptors = parsePlugin(containerLaunchers,
[20/51] [abbrv] tez git commit: TEZ-2361. Propagate dag completion to
TaskCommunicator. (sseth)
Posted by ss...@apache.org.
TEZ-2361. Propagate dag completion to TaskCommunicator. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/9d38581a
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/9d38581a
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/9d38581a
Branch: refs/heads/TEZ-2003
Commit: 9d38581a3f73b262b413808774ebe6160b67538e
Parents: e8c0c7a
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 23 17:26:25 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../apache/tez/dag/api/TaskCommunicator.java | 12 +++-
.../org/apache/tez/dag/app/DAGAppMaster.java | 4 +-
.../dag/app/TaskAttemptListenerImpTezDag.java | 17 +++++-
.../dag/app/TaskCommunicatorContextImpl.java | 64 +++++++++++++++++---
.../tez/dag/app/TezTaskCommunicatorImpl.java | 5 ++
.../tez/dag/app/launcher/ContainerLauncher.java | 3 -
.../dag/app/launcher/ContainerLauncherImpl.java | 12 ----
.../app/launcher/ContainerLauncherRouter.java | 10 +++
.../app/launcher/LocalContainerLauncher.java | 9 ---
.../apache/tez/dag/app/MockDAGAppMaster.java | 11 ----
.../rm/TezTestServiceTaskSchedulerService.java | 2 +-
.../TezTestServiceTaskCommunicatorImpl.java | 2 +-
.../tez/tests/TestExternalTezServices.java | 2 +
14 files changed, 103 insertions(+), 51 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 7c13110..f6bc8e7 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -17,5 +17,6 @@ ALL CHANGES:
TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
TEZ-2347. Expose additional information in TaskCommunicatorContext.
+ TEZ-2361. Propagate dag completion to TaskCommunicator.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index a2cd858..cadca0c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -14,7 +14,6 @@
package org.apache.tez.dag.api;
-import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Map;
@@ -74,4 +73,15 @@ public abstract class TaskCommunicator extends AbstractService {
* @throws Exception
*/
public abstract void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception;
+
+ /**
+ * Indicates the current running dag is complete. The TaskCommunicatorContext can be used to
+ * query information about the current dag during the duration of the dagComplete invocation.
+ *
+ * After this, the contents returned from querying the context may change at any point - due to
+ * the next dag being submitted.
+ */
+ // TODO TEZ-2003 This is extremely difficult to use. Add the dagStarted notification, and potentially
+ // throw exceptions between a dagComplete and dagStart invocation.
+ public abstract void dagComplete(String dagName);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 6ab0f8e..04e72db 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -766,7 +766,7 @@ public class DAGAppMaster extends AbstractService {
DAGAppMasterEventDagCleanup cleanupEvent = (DAGAppMasterEventDagCleanup) event;
LOG.info("Cleaning up DAG: name=" + cleanupEvent.getDag().getName() + ", with id=" +
cleanupEvent.getDag().getID());
- containerLauncher.dagComplete(cleanupEvent.getDag());
+ containerLauncherRouter.dagComplete(cleanupEvent.getDag());
taskAttemptListener.dagComplete(cleanupEvent.getDag());
nodes.dagComplete(cleanupEvent.getDag());
containers.dagComplete(cleanupEvent.getDag());
@@ -780,7 +780,7 @@ public class DAGAppMaster extends AbstractService {
case NEW_DAG_SUBMITTED:
// Inform sub-components that a new DAG has been submitted.
taskSchedulerEventHandler.dagSubmitted();
- containerLauncher.dagSubmitted();
+ containerLauncherRouter.dagSubmitted();
taskAttemptListener.dagSubmitted();
break;
default:
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 386e4af..7cdf292 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -80,6 +80,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
private final AppContext context;
private final TaskCommunicator[] taskCommunicators;
+ private final TaskCommunicatorContext[] taskCommunicatorContexts;
protected final TaskHeartbeatHandler taskHeartbeatHandler;
protected final ContainerHeartbeatHandler containerHeartbeatHandler;
@@ -122,7 +123,9 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
}
this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
+ this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorClassIdentifiers.length];
for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
+ taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, i);
taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
}
// TODO TEZ-2118 Start using taskCommunicator indices properly
@@ -147,10 +150,10 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
LOG.info("Using Default Task Communicator");
- return new TezTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+ return new TezTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
} else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Using Default Local Task Communicator");
- return new TezLocalTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+ return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
} else {
LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
@@ -158,7 +161,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
try {
Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
ctor.setAccessible(true);
- return ctor.newInstance(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+ return ctor.newInstance(taskCommunicatorContexts[taskCommIndex]);
} catch (NoSuchMethodException e) {
throw new TezUncheckedException(e);
} catch (InvocationTargetException e) {
@@ -317,6 +320,14 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
// This becomes more relevant when task kills without container kills are allowed.
// TODO TEZ-2336. Send a signal to containers indicating DAG completion.
+
+ // Inform all communicators of the dagCompletion.
+ for (int i = 0 ; i < taskCommunicators.length ; i++) {
+ ((TaskCommunicatorContextImpl)taskCommunicatorContexts[i]).dagCompleteStart(dag);
+ taskCommunicators[i].dagComplete(dag.getName());
+ ((TaskCommunicatorContextImpl)taskCommunicatorContexts[i]).dagCompleteEnd();
+ }
+
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index 4cb0c93..790066f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -17,6 +17,11 @@ package org.apache.tez.dag.app;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
@@ -33,6 +38,7 @@ import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.event.VertexState;
import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.Vertex;
import org.apache.tez.dag.app.dag.VertexStateUpdateListener;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -44,6 +50,10 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
private final AppContext context;
private final TaskAttemptListenerImpTezDag taskAttemptListener;
private final int taskCommunicatorIndex;
+ private final ReentrantReadWriteLock.ReadLock dagChangedReadLock;
+ private final ReentrantReadWriteLock.WriteLock dagChangedWriteLock;
+
+ private DAG dag;
public TaskCommunicatorContextImpl(AppContext appContext,
TaskAttemptListenerImpTezDag taskAttemptListener,
@@ -51,6 +61,10 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
this.context = appContext;
this.taskAttemptListener = taskAttemptListener;
this.taskCommunicatorIndex = taskCommunicatorIndex;
+
+ ReentrantReadWriteLock dagChangedLock = new ReentrantReadWriteLock();
+ dagChangedReadLock = dagChangedLock.readLock();
+ dagChangedWriteLock = dagChangedLock.writeLock();
}
@Override
@@ -111,18 +125,19 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
public void registerForVertexStateUpdates(String vertexName,
@Nullable Set<VertexState> stateSet) {
Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
- context.getCurrentDAG().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet, this);
+ getDag().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet,
+ this);
}
@Override
public String getCurretnDagName() {
- return context.getCurrentDAG().getName();
+ return getDag().getName();
}
@Override
public Iterable<String> getInputVertexNames(String vertexName) {
Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
- Vertex vertex = context.getCurrentDAG().getVertex(vertexName);
+ Vertex vertex = getDag().getVertex(vertexName);
Set<Vertex> sources = vertex.getInputVertices().keySet();
return Iterables.transform(sources, new Function<Vertex, String>() {
@Override
@@ -135,31 +150,32 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
@Override
public int getVertexTotalTaskCount(String vertexName) {
Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
- return context.getCurrentDAG().getVertex(vertexName).getTotalTasks();
+ return getDag().getVertex(vertexName).getTotalTasks();
}
@Override
public int getVertexCompletedTaskCount(String vertexName) {
Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
- return context.getCurrentDAG().getVertex(vertexName).getCompletedTasks();
+ return getDag().getVertex(vertexName).getCompletedTasks();
}
@Override
public int getVertexRunningTaskCount(String vertexName) {
Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
- return context.getCurrentDAG().getVertex(vertexName).getRunningTasks();
+ return getDag().getVertex(vertexName).getRunningTasks();
}
@Override
public long getFirstAttemptStartTime(String vertexName, int taskIndex) {
Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
Preconditions.checkArgument(taskIndex >=0, "TaskIndex must be > 0");
- return context.getCurrentDAG().getVertex(vertexName).getTask(taskIndex).getFirstAttemptStartTime();
+ return getDag().getVertex(vertexName).getTask(
+ taskIndex).getFirstAttemptStartTime();
}
@Override
public long getDagStartTime() {
- return context.getCurrentDAG().getStartTime();
+ return getDag().getStartTime();
}
@Override
@@ -171,4 +187,36 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
throw new TezUncheckedException(e);
}
}
+
+ private DAG getDag() {
+ dagChangedReadLock.lock();
+ try {
+ if (dag != null) {
+ return dag;
+ } else {
+ return context.getCurrentDAG();
+ }
+ } finally {
+ dagChangedReadLock.unlock();
+ }
+ }
+
+ @InterfaceAudience.Private
+ public void dagCompleteStart(DAG dag) {
+ dagChangedWriteLock.lock();
+ try {
+ this.dag = dag;
+ } finally {
+ dagChangedWriteLock.unlock();
+ }
+ }
+
+ public void dagCompleteEnd() {
+ dagChangedWriteLock.lock();
+ try {
+ this.dag = null;
+ } finally {
+ dagChangedWriteLock.unlock();
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index fa2749a..6200a5b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -260,6 +260,11 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
// Empty. Not registering, or expecting any updates.
}
+ @Override
+ public void dagComplete(String dagName) {
+ // Nothing to do at the moment. Some of the TODOs from TaskAttemptListener apply here.
+ }
+
protected String getTokenIdentifier() {
return tokenIdentifier;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
index 8a8498f..ea07a1d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
@@ -26,7 +26,4 @@ import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
public interface ContainerLauncher
extends EventHandler<NMCommunicatorEvent> {
- void dagComplete(DAG dag);
-
- void dagSubmitted();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
index 94889a1..a1eb2a7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
@@ -30,7 +30,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.tez.dag.app.dag.DAG;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -111,17 +110,6 @@ public class ContainerLauncherImpl extends AbstractService implements
}
}
- @Override
- public void dagComplete(DAG dag) {
- // Nothing required at the moment. Containers are shared across DAGs
- }
-
- @Override
- public void dagSubmitted() {
- // Nothing to do right now. Indicates that a new DAG has been submitted and
- // the context has updated information.
- }
-
private static enum ContainerState {
PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index dd3571e..db145f4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -27,6 +27,7 @@ import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -128,6 +129,15 @@ public class ContainerLauncherRouter extends AbstractService
}
}
+ public void dagComplete(DAG dag) {
+ // Nothing required at the moment. Containers are shared across DAGs
+ }
+
+ public void dagSubmitted() {
+ // Nothing to do right now. Indicates that a new DAG has been submitted and
+ // the context has updated information.
+ }
+
@Override
public void handle(NMCommunicatorEvent event) {
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index 7dbf937..fe23409 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -44,7 +44,6 @@ import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.tez.dag.app.dag.DAG;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -170,14 +169,6 @@ public class LocalContainerLauncher extends AbstractService implements
callbackExecutor.shutdownNow();
}
- @Override
- public void dagComplete(DAG dag) {
- }
-
- @Override
- public void dagSubmitted() {
- }
-
// Thread to monitor the queue of incoming NMCommunicator events
private class TezSubTaskRunner implements Runnable {
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index b39eee2..9882954 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -34,10 +34,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
-import org.apache.tez.dag.app.dag.DAG;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.service.AbstractService;
@@ -54,7 +52,6 @@ import org.apache.tez.common.counters.TezCounters;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.launcher.ContainerLauncher;
import org.apache.tez.dag.app.launcher.ContainerLauncherRouter;
@@ -150,14 +147,6 @@ public class MockDAGAppMaster extends DAGAppMaster {
this.goFlag = goFlag;
}
- @Override
- public void dagComplete(DAG dag) {
- }
-
- @Override
- public void dagSubmitted() {
- }
-
public class ContainerData {
ContainerId cId;
TezTaskAttemptID taId;
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 50dfb24..073cb50 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -158,7 +158,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
}
@Override
- public void resetMatchLocalityForAllHeldContainers() {
+ public void dagComplete() {
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index ef983c2..cf28b11 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -150,7 +150,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
t = se.getCause();
}
if (t instanceof RemoteException) {
- RemoteException re = (RemoteException)t;
+ RemoteException re = (RemoteException) t;
String message = re.toString();
if (message.contains(RejectedExecutionException.class.getName())) {
getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
http://git-wip-us.apache.org/repos/asf/tez/blob/9d38581a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 4d0a610..45c70f1 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -39,6 +39,7 @@ import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
import org.apache.tez.examples.HashJoinExample;
import org.apache.tez.examples.JoinDataGen;
import org.apache.tez.examples.JoinValidateConfigured;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
import org.apache.tez.runtime.library.processor.SleepProcessor;
import org.apache.tez.service.MiniTezTestServiceCluster;
import org.apache.tez.service.impl.ContainerRunnerImpl;
@@ -124,6 +125,7 @@ public class TestExternalTezServices {
remoteFs.mkdirs(stagingDirPath);
// This is currently configured to push tasks into the Service, and then use the standard RPC
confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
+ confForJobs.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false);
confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
[17/51] [abbrv] tez git commit: TEZ-2285. Allow TaskCommunicators to
indicate task/container liveness. (sseth)
Posted by ss...@apache.org.
TEZ-2285. Allow TaskCommunicators to indicate task/container liveness. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/b4ed5612
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/b4ed5612
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/b4ed5612
Branch: refs/heads/TEZ-2003
Commit: b4ed56123eb25d7007708cfbcd8406efd6f3f966
Parents: 08a196a
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:22:09 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../apache/tez/dag/api/TaskCommunicatorContext.java | 4 ++++
.../tez/dag/app/TaskAttemptListenerImpTezDag.java | 10 ++++++++++
.../apache/tez/dag/app/TezTaskCommunicatorImpl.java | 16 +++++++++-------
4 files changed, 24 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/b4ed5612/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index e2c428d..9d6b220 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -14,5 +14,6 @@ ALL CHANGES:
TEZ-2241. Miscellaneous fixes after last reabse.
TEZ-2283. Fixes after rebase 04/07.
TEZ-2284. Separate TaskReporter into an interface.
+ TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/b4ed5612/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index a85fb7f..0c3bac3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -43,6 +43,10 @@ public interface TaskCommunicatorContext {
boolean isKnownContainer(ContainerId containerId);
+ void taskAlive(TezTaskAttemptID taskAttemptId);
+
+ void containerAlive(ContainerId containerId);
+
// TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
http://git-wip-us.apache.org/repos/asf/tez/blob/b4ed5612/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 3798b6f..a6994d2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -258,6 +258,16 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
@Override
+ public void taskAlive(TezTaskAttemptID taskAttemptId) {
+ taskHeartbeatHandler.pinged(taskAttemptId);
+ }
+
+ @Override
+ public void containerAlive(ContainerId containerId) {
+ pingContainerHeartbeatHandler(containerId);
+ }
+
+ @Override
public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
context.getEventHandler()
.handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
http://git-wip-us.apache.org/repos/asf/tez/blob/b4ed5612/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index bba06fd..a4a707b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -65,17 +65,19 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
null, true, null, null, false);
private final TaskCommunicatorContext taskCommunicatorContext;
+ private final TezTaskUmbilicalProtocol taskUmbilical;
- private final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
+ protected final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
new ConcurrentHashMap<ContainerId, ContainerInfo>();
- private final ConcurrentMap<TaskAttempt, ContainerId> attemptToContainerMap =
+ protected final ConcurrentMap<TaskAttempt, ContainerId> attemptToContainerMap =
new ConcurrentHashMap<TaskAttempt, ContainerId>();
- private final TezTaskUmbilicalProtocol taskUmbilical;
- private final String tokenIdentifier;
- private final Token<JobTokenIdentifier> sessionToken;
+
+ protected final String tokenIdentifier;
+ protected final Token<JobTokenIdentifier> sessionToken;
protected InetSocketAddress address;
- private Server server;
+
+ protected volatile Server server;
public static final class ContainerInfo {
@@ -440,7 +442,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
// Holder for Task information, which eventually will likely be VertexImplm taskIndex, attemptIndex
- private static class TaskAttempt {
+ protected static class TaskAttempt {
// TODO TEZ-2003 Change this to work with VertexName, int id, int version
// TODO TEZ-2003 Avoid constructing this unit all over the place
private TezTaskAttemptID taskAttemptId;
[41/51] [abbrv] tez git commit: TEZ-2495. Inform TaskCommunicaor
about Task and Container termination reasons. (sseth)
Posted by ss...@apache.org.
TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/b41dbb05
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/b41dbb05
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/b41dbb05
Branch: refs/heads/TEZ-2003
Commit: b41dbb054370d561157aa2ccf0b932948ddb4a35
Parents: a519c29
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu May 28 02:01:04 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/common/TezUtilsInternal.java | 31 ++++++++-----
.../apache/tez/dag/api/ContainerEndReason.java | 27 +++++++++++
.../tez/dag/api/TaskAttemptEndReason.java | 13 +++---
.../apache/tez/dag/api/TaskCommunicator.java | 11 +++--
.../apache/tez/dag/app/TaskAttemptListener.java | 6 ++-
.../dag/app/TaskAttemptListenerImpTezDag.java | 9 ++--
.../tez/dag/app/TezTaskCommunicatorImpl.java | 6 ++-
.../rm/container/AMContainerEventCompleted.java | 41 +++++++++++++++++
.../dag/app/rm/container/AMContainerImpl.java | 35 ++++++++-------
.../app/TestTaskAttemptListenerImplTezDag.java | 8 ++--
.../dag/app/rm/container/TestAMContainer.java | 47 +++++++++++---------
.../TezTestServiceTaskCommunicatorImpl.java | 9 ++--
.../apache/tez/runtime/task/TezTaskRunner2.java | 20 +++++++--
14 files changed, 186 insertions(+), 78 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index d651960..e333832 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -28,5 +28,6 @@ ALL CHANGES:
TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
TEZ-2465. Retrun the status of a kill request in TaskRunner2.
TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
+ TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
index 347a4f6..0bdeb79 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
@@ -246,10 +246,16 @@ public class TezUtilsInternal {
return TaskAttemptTerminationCause.COMMUNICATION_ERROR;
case SERVICE_BUSY:
return TaskAttemptTerminationCause.SERVICE_BUSY;
- case INTERRUPTED_BY_SYSTEM:
- return TaskAttemptTerminationCause.INTERRUPTED_BY_SYSTEM;
- case INTERRUPTED_BY_USER:
- return TaskAttemptTerminationCause.INTERRUPTED_BY_USER;
+ case INTERNAL_PREEMPTION:
+ return TaskAttemptTerminationCause.INTERNAL_PREEMPTION;
+ case EXTERNAL_PREEMPTION:
+ return TaskAttemptTerminationCause.EXTERNAL_PREEMPTION;
+ case APPLICATION_ERROR:
+ return TaskAttemptTerminationCause.APPLICATION_ERROR;
+ case FRAMEWORK_ERROR:
+ return TaskAttemptTerminationCause.FRAMEWORK_ERROR;
+ case NODE_FAILED:
+ return TaskAttemptTerminationCause.NODE_FAILED;
case OTHER:
return TaskAttemptTerminationCause.UNKNOWN_ERROR;
default:
@@ -267,20 +273,24 @@ public class TezUtilsInternal {
return TaskAttemptEndReason.COMMUNICATION_ERROR;
case SERVICE_BUSY:
return TaskAttemptEndReason.SERVICE_BUSY;
+ case INTERNAL_PREEMPTION:
+ return TaskAttemptEndReason.INTERNAL_PREEMPTION;
+ case EXTERNAL_PREEMPTION:
+ return TaskAttemptEndReason.EXTERNAL_PREEMPTION;
+ case APPLICATION_ERROR:
+ return TaskAttemptEndReason.APPLICATION_ERROR;
+ case FRAMEWORK_ERROR:
+ return TaskAttemptEndReason.FRAMEWORK_ERROR;
+ case NODE_FAILED:
+ return TaskAttemptEndReason.NODE_FAILED;
case INTERRUPTED_BY_SYSTEM:
- return TaskAttemptEndReason.INTERRUPTED_BY_SYSTEM;
case INTERRUPTED_BY_USER:
- return TaskAttemptEndReason.INTERRUPTED_BY_USER;
case UNKNOWN_ERROR:
case TERMINATED_BY_CLIENT:
case TERMINATED_AT_SHUTDOWN:
- case INTERNAL_PREEMPTION:
- case EXTERNAL_PREEMPTION:
case TERMINATED_INEFFECTIVE_SPECULATION:
case TERMINATED_EFFECTIVE_SPECULATION:
case TERMINATED_ORPHANED:
- case APPLICATION_ERROR:
- case FRAMEWORK_ERROR:
case INPUT_READ_ERROR:
case OUTPUT_WRITE_ERROR:
case OUTPUT_LOST:
@@ -288,7 +298,6 @@ public class TezUtilsInternal {
case CONTAINER_LAUNCH_FAILED:
case CONTAINER_EXITED:
case CONTAINER_STOPPED:
- case NODE_FAILED:
case NODE_DISK_ERROR:
default:
return TaskAttemptEndReason.OTHER;
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
new file mode 100644
index 0000000..e13e886
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
@@ -0,0 +1,27 @@
+/*
+ * 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.tez.dag.api;
+
+// TODO TEZ-2003 Expose as a public API
+public enum ContainerEndReason {
+ NODE_FAILED, // Completed because the node running the container was marked as dead
+ INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
+ EXTERNAL_PREEMPTION, // Preempted due to cluster contention
+ APPLICATION_ERROR, // An error in the AM caused by user code
+ FRAMEWORK_ERROR, // An error in the AM - likely a bug.
+ LAUNCH_FAILED, // Failure to launch the container
+ COMPLETED, // Completed via normal flow
+ OTHER
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
index 96a4768..de78d21 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
@@ -16,9 +16,12 @@ package org.apache.tez.dag.api;
// TODO TEZ-2003 Expose as a public API
public enum TaskAttemptEndReason {
- COMMUNICATION_ERROR,
- SERVICE_BUSY,
- INTERRUPTED_BY_SYSTEM,
- INTERRUPTED_BY_USER,
- OTHER
+ NODE_FAILED, // Completed because the node running the container was marked as dead
+ COMMUNICATION_ERROR, // Communication error with the task
+ SERVICE_BUSY, // External service busy
+ INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
+ EXTERNAL_PREEMPTION, // Preempted due to cluster contention
+ APPLICATION_ERROR, // An error in the AM caused by user code
+ FRAMEWORK_ERROR, // An error in the AM - likely a bug.
+ OTHER // Unknown reason
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index 2651013..d0a006b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -36,7 +36,10 @@ public abstract class TaskCommunicator extends AbstractService {
public abstract void registerRunningContainer(ContainerId containerId, String hostname, int port);
// TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
- public abstract void registerContainerEnd(ContainerId containerId);
+ public abstract void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason);
+
+ // TODO TEZ-2003 Provide additional inforamtion like reason for container end / Task end.
+ // Was it caused by preemption - or as a result of a general task completion / container completion
// TODO TEZ-2003 TaskSpec breakup into a clean interface
// TODO TEZ-2003 Add support for priority
@@ -48,11 +51,7 @@ public abstract class TaskCommunicator extends AbstractService {
// TODO TEZ-2003. Are additional APIs required to mark a container as completed ? - for completeness.
// TODO TEZ-2003 Remove reference to TaskAttemptID
- // TODO TEZ-2003 This needs some information about why the attempt is being unregistered.
- // e.g. preempted in which case the task may need to be informed. Alternately as a result of
- // a failed task.
- // In case of preemption - a killTask API is likely a better bet than trying to overload this method.
- public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
+ public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason);
// TODO TEZ-2003 This doesn't necessarily belong here. A server may not start within the AM.
public abstract InetSocketAddress getAddress();
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
index e4dad27..92e38ae 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
@@ -21,6 +21,8 @@ package org.apache.tez.dag.app;
import java.net.InetSocketAddress;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.app.rm.container.AMContainerTask;
@@ -34,9 +36,9 @@ public interface TaskAttemptListener {
void registerTaskAttempt(AMContainerTask amContainerTask, ContainerId containerId, int taskCommId);
- void unregisterRunningContainer(ContainerId containerId, int taskCommId);
+ void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason);
- void unregisterTaskAttempt(TezTaskAttemptID attemptID, int taskCommId);
+ void unregisterTaskAttempt(TezTaskAttemptID attemptID, int taskCommId, TaskAttemptEndReason endReason);
void dagComplete(DAG dag);
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index db78fa9..1c61a0d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentMap;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.collections4.ListUtils;
+import org.apache.tez.dag.api.ContainerEndReason;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
@@ -355,7 +356,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
@Override
- public void unregisterRunningContainer(ContainerId containerId, int taskCommId) {
+ public void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason) {
if (LOG.isDebugEnabled()) {
LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId);
}
@@ -363,7 +364,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
if (containerInfo.taskAttemptId != null) {
registeredAttempts.remove(containerInfo.taskAttemptId);
}
- taskCommunicators[taskCommId].registerContainerEnd(containerId);
+ taskCommunicators[taskCommId].registerContainerEnd(containerId, endReason);
}
@Override
@@ -404,7 +405,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
@Override
- public void unregisterTaskAttempt(TezTaskAttemptID attemptId, int taskCommId) {
+ public void unregisterTaskAttempt(TezTaskAttemptID attemptId, int taskCommId, TaskAttemptEndReason endReason) {
ContainerId containerId = registeredAttempts.remove(attemptId);
if (containerId == null) {
LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
@@ -418,7 +419,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
// Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
registeredContainers.put(containerId, NULL_CONTAINER_INFO);
- taskCommunicators[taskCommId].unregisterRunningTaskAttempt(attemptId);
+ taskCommunicators[taskCommId].unregisterRunningTaskAttempt(attemptId, endReason);
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index accde2c..3774eb4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -41,6 +41,8 @@ import org.apache.tez.common.ContainerContext;
import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -185,7 +187,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
}
@Override
- public void registerContainerEnd(ContainerId containerId) {
+ public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
ContainerInfo containerInfo = registeredContainers.remove(containerId);
if (containerInfo != null) {
synchronized(containerInfo) {
@@ -231,7 +233,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
@Override
- public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
+ public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason) {
TaskAttempt taskAttempt = new TaskAttempt(taskAttemptID);
ContainerId containerId = attemptToContainerMap.remove(taskAttempt);
if(containerId == null) {
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
index 9bb6d7f..8ef2a83 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventCompleted.java
@@ -20,6 +20,7 @@ package org.apache.tez.dag.app.rm.container;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.ContainerEndReason;
import org.apache.tez.dag.records.TaskAttemptTerminationCause;
public class AMContainerEventCompleted extends AMContainerEvent {
@@ -61,4 +62,44 @@ public class AMContainerEventCompleted extends AMContainerEvent {
return errCause;
}
+ public ContainerEndReason getContainerEndReason() {
+ if (errCause != null) {
+ switch (errCause) {
+ case INTERNAL_PREEMPTION:
+ return ContainerEndReason.INTERNAL_PREEMPTION;
+ case EXTERNAL_PREEMPTION:
+ return ContainerEndReason.EXTERNAL_PREEMPTION;
+ case FRAMEWORK_ERROR:
+ return ContainerEndReason.FRAMEWORK_ERROR;
+ case APPLICATION_ERROR:
+ return ContainerEndReason.APPLICATION_ERROR;
+ case CONTAINER_LAUNCH_FAILED:
+ return ContainerEndReason.LAUNCH_FAILED;
+ case NODE_FAILED:
+ return ContainerEndReason.NODE_FAILED;
+ case CONTAINER_EXITED:
+ return ContainerEndReason.COMPLETED;
+ case UNKNOWN_ERROR:
+ case TERMINATED_BY_CLIENT:
+ case TERMINATED_AT_SHUTDOWN:
+ case TERMINATED_INEFFECTIVE_SPECULATION:
+ case TERMINATED_EFFECTIVE_SPECULATION:
+ case TERMINATED_ORPHANED:
+ case INPUT_READ_ERROR:
+ case OUTPUT_WRITE_ERROR:
+ case OUTPUT_LOST:
+ case TASK_HEARTBEAT_ERROR:
+ case CONTAINER_STOPPED:
+ case NODE_DISK_ERROR:
+ case COMMUNICATION_ERROR:
+ case SERVICE_BUSY:
+ case INTERRUPTED_BY_SYSTEM:
+ case INTERRUPTED_BY_USER:
+ default:
+ return ContainerEndReason.OTHER;
+ }
+ } else {
+ return ContainerEndReason.OTHER;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index 39df2e8..e9e0f04 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -27,6 +27,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.security.Credentials;
@@ -624,7 +627,7 @@ public class AMContainerImpl implements AMContainer {
container.sendTerminatingToTaskAttempt(container.currentAttempt,
event.getMessage(), TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED);
}
- container.unregisterFromTAListener();
+ container.unregisterFromTAListener(ContainerEndReason.LAUNCH_FAILED);
container.deAllocate();
}
}
@@ -654,7 +657,7 @@ public class AMContainerImpl implements AMContainer {
}
container.containerLocalResources = null;
container.additionalLocalResources = null;
- container.unregisterFromTAListener();
+ container.unregisterFromTAListener(event.getContainerEndReason());
String diag = event.getDiagnostics();
if (!(diag == null || diag.equals(""))) {
LOG.info("Container " + container.getContainerId()
@@ -680,7 +683,7 @@ public class AMContainerImpl implements AMContainer {
container.sendTerminatingToTaskAttempt(container.currentAttempt,
getMessage(container, cEvent), TaskAttemptTerminationCause.CONTAINER_STOPPED);
}
- container.unregisterFromTAListener();
+ container.unregisterFromTAListener(ContainerEndReason.OTHER);
container.logStopped(container.currentAttempt == null ?
ContainerExitStatus.SUCCESS
: ContainerExitStatus.INVALID);
@@ -732,7 +735,7 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
super.transition(container, cEvent);
- container.unregisterFromTAListener();
+ container.unregisterFromTAListener(ContainerEndReason.NODE_FAILED);
container.deAllocate();
}
}
@@ -749,7 +752,7 @@ public class AMContainerImpl implements AMContainer {
container.getState(), TaskAttemptTerminationCause.FRAMEWORK_ERROR);
}
container.logStopped(ContainerExitStatus.ABORTED);
- container.unregisterFromTAListener();
+ container.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR);
container.sendStopRequestToNM();
}
}
@@ -821,7 +824,7 @@ public class AMContainerImpl implements AMContainer {
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
AMContainerEventAssignTA event = (AMContainerEventAssignTA) cEvent;
- container.unregisterAttemptFromListener(container.currentAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.FRAMEWORK_ERROR);
container.handleExtraTAAssign(event, container.currentAttempt);
}
}
@@ -832,7 +835,7 @@ public class AMContainerImpl implements AMContainer {
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
container.lastTaskFinishTime = System.currentTimeMillis();
container.completedAttempts.add(container.currentAttempt);
- container.unregisterAttemptFromListener(container.currentAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.OTHER);
container.currentAttempt = null;
}
}
@@ -849,7 +852,7 @@ public class AMContainerImpl implements AMContainer {
container.sendTerminatedToTaskAttempt(container.currentAttempt,
getMessage(container, event), event.getTerminationCause());
}
- container.unregisterAttemptFromListener(container.currentAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt, TezUtilsInternal.toTaskAttemptEndReason(event.getTerminationCause()));
container.registerFailedAttempt(container.currentAttempt);
container.currentAttempt= null;
super.transition(container, cEvent);
@@ -859,7 +862,7 @@ public class AMContainerImpl implements AMContainer {
protected static class StopRequestAtRunningTransition
extends StopRequestAtIdleTransition {
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
- container.unregisterAttemptFromListener(container.currentAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.OTHER);
super.transition(container, cEvent);
}
}
@@ -880,7 +883,7 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
super.transition(container, cEvent);
- container.unregisterAttemptFromListener(container.currentAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.NODE_FAILED);
}
}
@@ -889,7 +892,7 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
super.transition(container, cEvent);
- container.unregisterAttemptFromListener(container.currentAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt, TaskAttemptEndReason.FRAMEWORK_ERROR);
container.sendTerminatingToTaskAttempt(container.currentAttempt,
"Container " + container.getContainerId() +
" hit an invalid transition - " + cEvent.getType() + " at " +
@@ -1015,7 +1018,7 @@ public class AMContainerImpl implements AMContainer {
LOG.warn(errorMessage);
this.logStopped(ContainerExitStatus.INVALID);
this.sendStopRequestToNM();
- this.unregisterFromTAListener();
+ this.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR);
this.unregisterFromContainerListener();
}
@@ -1073,8 +1076,8 @@ public class AMContainerImpl implements AMContainer {
container.getNodeId(), container.getContainerToken(), launcherId));
}
- protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId) {
- taskAttemptListener.unregisterTaskAttempt(attemptId, taskCommId);
+ protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId, TaskAttemptEndReason endReason) {
+ taskAttemptListener.unregisterTaskAttempt(attemptId, taskCommId, endReason);
}
protected void registerAttemptWithListener(AMContainerTask amContainerTask) {
@@ -1085,8 +1088,8 @@ public class AMContainerImpl implements AMContainer {
taskAttemptListener.registerRunningContainer(containerId, taskCommId);
}
- protected void unregisterFromTAListener() {
- this.taskAttemptListener.unregisterRunningContainer(containerId, taskCommId);
+ protected void unregisterFromTAListener(ContainerEndReason endReason) {
+ this.taskAttemptListener.unregisterRunningContainer(containerId, taskCommId, endReason);
}
protected void registerWithContainerListener() {
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 34b9792..68d3baf 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -47,6 +47,8 @@ import org.apache.tez.common.ContainerContext;
import org.apache.tez.common.ContainerTask;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezException;
@@ -163,12 +165,12 @@ public class TestTaskAttemptListenerImplTezDag {
assertEquals(taskSpec, containerTask.getTaskSpec());
// Task unregistered. Should respond to heartbeats
- taskAttemptListener.unregisterTaskAttempt(taskAttemptID, 0);
+ taskAttemptListener.unregisterTaskAttempt(taskAttemptID, 0, TaskAttemptEndReason.OTHER);
containerTask = tezUmbilical.getTask(containerContext2);
assertNull(containerTask);
// Container unregistered. Should send a shouldDie = true
- taskAttemptListener.unregisterRunningContainer(containerId2, 0);
+ taskAttemptListener.unregisterRunningContainer(containerId2, 0, ContainerEndReason.OTHER);
containerTask = tezUmbilical.getTask(containerContext2);
assertTrue(containerTask.shouldDie());
@@ -182,7 +184,7 @@ public class TestTaskAttemptListenerImplTezDag {
doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false, 0);
taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3, 0);
- taskAttemptListener.unregisterRunningContainer(containerId3, 0);
+ taskAttemptListener.unregisterRunningContainer(containerId3, 0, ContainerEndReason.OTHER);
containerTask = tezUmbilical.getTask(containerContext3);
assertTrue(containerTask.shouldDie());
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index bdd0f61..b8b4774 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -63,6 +63,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerHeartbeatHandler;
@@ -132,14 +134,14 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
assertNull(wc.amContainer.getCurrentTaskAttempt());
- verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
+ verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
// Container completed
wc.containerCompleted();
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
verify(wc.chh).unregister(wc.containerID);
assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -181,13 +183,13 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
assertNull(wc.amContainer.getCurrentTaskAttempt());
- verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
+ verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
wc.containerCompleted();
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
verify(wc.chh).unregister(wc.containerID);
assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -232,7 +234,7 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
assertNull(wc.amContainer.getCurrentTaskAttempt());
- verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
+ verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0, TaskAttemptEndReason.OTHER);
TezTaskAttemptID taId2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taId2);
@@ -247,14 +249,14 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
assertNull(wc.amContainer.getCurrentTaskAttempt());
- verify(wc.tal).unregisterTaskAttempt(taId2, 0);
+ verify(wc.tal).unregisterTaskAttempt(taId2, 0, TaskAttemptEndReason.OTHER);
// Container completed
wc.containerCompleted();
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
verify(wc.chh).unregister(wc.containerID);
assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
@@ -287,7 +289,7 @@ public class TestAMContainer {
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
verify(wc.chh).unregister(wc.containerID);
assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -324,7 +326,7 @@ public class TestAMContainer {
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
verify(wc.chh).unregister(wc.containerID);
assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -347,7 +349,7 @@ public class TestAMContainer {
wc.assignTaskAttempt(taID2);
wc.verifyState(AMContainerState.STOP_REQUESTED);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.FRAMEWORK_ERROR);
verify(wc.chh).unregister(wc.containerID);
// 1 for NM stop request. 2 TERMINATING to TaskAttempt.
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -385,7 +387,7 @@ public class TestAMContainer {
wc.assignTaskAttempt(taID2);
wc.verifyState(AMContainerState.STOP_REQUESTED);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.FRAMEWORK_ERROR);
verify(wc.chh).unregister(wc.containerID);
// 1 for NM stop request. 2 TERMINATING to TaskAttempt.
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -421,7 +423,7 @@ public class TestAMContainer {
wc.containerTimedOut();
wc.verifyState(AMContainerState.STOP_REQUESTED);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
verify(wc.chh).unregister(wc.containerID);
// 1 to TA, 1 for RM de-allocate.
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -455,7 +457,7 @@ public class TestAMContainer {
wc.stopRequest();
wc.verifyState(AMContainerState.STOP_REQUESTED);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
verify(wc.chh).unregister(wc.containerID);
// 1 to TA, 1 for RM de-allocate.
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -489,7 +491,7 @@ public class TestAMContainer {
wc.launchFailed();
wc.verifyState(AMContainerState.STOPPING);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.LAUNCH_FAILED);
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -539,7 +541,7 @@ public class TestAMContainer {
wc.containerCompleted();
wc.verifyState(AMContainerState.COMPLETED);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -569,7 +571,7 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
wc.verifyState(AMContainerState.COMPLETED);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -600,7 +602,7 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.ABORTED, TaskAttemptTerminationCause.NODE_FAILED);
wc.verifyState(AMContainerState.COMPLETED);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.NODE_FAILED);
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -631,7 +633,7 @@ public class TestAMContainer {
wc.containerCompleted();
wc.verifyState(AMContainerState.COMPLETED);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -660,7 +662,7 @@ public class TestAMContainer {
wc.containerCompleted();
wc.verifyState(AMContainerState.COMPLETED);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.COMPLETED);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -695,7 +697,7 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.PREEMPTED, TaskAttemptTerminationCause.EXTERNAL_PREEMPTION);
wc.verifyState(AMContainerState.COMPLETED);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.EXTERNAL_PREEMPTION);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -732,7 +734,8 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.INVALID, TaskAttemptTerminationCause.INTERNAL_PREEMPTION);
wc.verifyState(AMContainerState.COMPLETED);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0,
+ ContainerEndReason.INTERNAL_PREEMPTION);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -769,7 +772,7 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
wc.verifyState(AMContainerState.COMPLETED);
verify(wc.tal).registerRunningContainer(wc.containerID, 0);
- verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0, ContainerEndReason.OTHER);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index cf28b11..98673a6 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.ContainerEndReason;
import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
@@ -98,8 +99,8 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
}
@Override
- public void registerContainerEnd(ContainerId containerId) {
- super.registerContainerEnd(containerId);
+ public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
+ super.registerContainerEnd(containerId, endReason);
}
@Override
@@ -175,8 +176,8 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
}
@Override
- public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
- super.unregisterRunningTaskAttempt(taskAttemptID);
+ public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason) {
+ super.unregisterRunningTaskAttempt(taskAttemptID, endReason);
// Nothing else to do for now. The push API in the test does not support termination of a running task
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b41dbb05/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index 3bf9f84..15629fd 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -261,7 +261,13 @@ public class TezTaskRunner2 {
taskRunnerCallable.interruptTask();
}
return true;
+ } else {
+ LOG.info("Ignoring killTask request for {} since end reason is already set to {}",
+ task.getTaskAttemptID(), firstEndReason);
}
+ } else {
+ LOG.info("Ignoring killTask request for {} since it is not in a running state",
+ task.getTaskAttemptID());
}
}
return false;
@@ -389,10 +395,18 @@ public class TezTaskRunner2 {
isFirstTerminate = trySettingEndReason(EndReason.CONTAINER_STOP_REQUESTED);
// Respect stopContainerRequested since it can come in at any point, despite a previous failure.
stopContainerRequested.set(true);
- }
- if (isFirstTerminate) {
- killTask();
+ if (isFirstTerminate) {
+ LOG.info("Attempting to abort {} since a shutdown request was received",
+ task.getTaskAttemptID());
+ if (taskRunnerCallable != null) {
+ taskKillStartTime = System.currentTimeMillis();
+ taskRunnerCallable.interruptTask();
+ }
+ } else {
+ LOG.info("Not acting on shutdown request for {} since the task is not in running state",
+ task.getTaskAttemptID());
+ }
}
}
}
[27/51] [abbrv] tez git commit: TEZ-2465. Retrun the status of a kill
request in TaskRunner2. (sseth)
Posted by ss...@apache.org.
TEZ-2465. Retrun the status of a kill request in TaskRunner2. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/db3f6aa7
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/db3f6aa7
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/db3f6aa7
Branch: refs/heads/TEZ-2003
Commit: db3f6aa793bc1e92ec0115cda90395bf1f7497de
Parents: 625220e
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue May 19 13:57:58 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../java/org/apache/tez/runtime/task/TezTaskRunner2.java | 8 +++++++-
2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/db3f6aa7/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index ed72d6b..ca3383c 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -26,5 +26,6 @@ ALL CHANGES:
TEZ-2438. tez-tools version in the branch is incorrect.
TEZ-2434. Allow tasks to be killed in the Runtime.
TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
+ TEZ-2465. Retrun the status of a kill request in TaskRunner2.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/db3f6aa7/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index ffbc6e8..3bf9f84 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -247,7 +247,11 @@ public class TezTaskRunner2 {
}
}
- public void killTask() {
+ /**
+ * Attempt to kill the running task, if it hasn't already completed for some other reason.
+ * @return true if the task kill was honored, false otherwise
+ */
+ public boolean killTask() {
synchronized (this) {
if (isRunningState()) {
if (trySettingEndReason(EndReason.KILL_REQUESTED)) {
@@ -256,9 +260,11 @@ public class TezTaskRunner2 {
taskKillStartTime = System.currentTimeMillis();
taskRunnerCallable.interruptTask();
}
+ return true;
}
}
}
+ return false;
}
[45/51] [abbrv] tez git commit: TEZ-2626. Fix log lines with DEBUG in
messages, consolidate TEZ-2003 TODOs. (sseth)
Posted by ss...@apache.org.
TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/4766b71c
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/4766b71c
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/4766b71c
Branch: refs/heads/TEZ-2003
Commit: 4766b71c0830d24d6c75834d99893307100885dd
Parents: 306020d
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Jul 30 13:39:40 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:58 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
pom.xml | 2 +-
.../serviceplugins/api/ContainerLauncher.java | 4 ---
.../tez/serviceplugins/api/TaskScheduler.java | 5 ++++
.../api/TaskSchedulerContext.java | 6 ++--
.../apache/tez/dag/api/TaskCommunicator.java | 29 +++++++-------------
.../tez/dag/api/TaskCommunicatorContext.java | 24 ++++++++--------
.../tez/dag/api/TaskHeartbeatRequest.java | 6 ++--
.../tez/dag/api/TaskHeartbeatResponse.java | 2 +-
.../dag/app/TaskAttemptListenerImpTezDag.java | 7 ++---
.../dag/app/TaskCommunicatorContextImpl.java | 2 +-
.../tez/dag/app/TezTaskCommunicatorImpl.java | 3 +-
.../apache/tez/dag/app/dag/impl/DAGImpl.java | 2 +-
.../tez/dag/app/dag/impl/TaskAttemptImpl.java | 2 +-
.../dag/app/rm/TaskSchedulerEventHandler.java | 6 ----
.../rm/container/AMContainerEventAssignTA.java | 2 --
tez-ext-service-tests/pom.xml | 1 -
.../TezTestServiceContainerLauncher.java | 4 +--
.../TezTestServiceNoOpContainerLauncher.java | 2 +-
.../rm/TezTestServiceTaskSchedulerService.java | 6 ++--
.../tez/service/impl/ContainerRunnerImpl.java | 6 ++--
.../tez/tests/TestExternalTezServices.java | 2 --
.../internals/api/TaskReporterInterface.java | 4 +--
23 files changed, 51 insertions(+), 77 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9b3967a..c7a3dcc 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -41,5 +41,6 @@ ALL CHANGES:
TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
TEZ-2441. Add tests for TezTaskRunner2.
TEZ-2657. Add tests for client side changes - specifying plugins, etc.
+ TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1ced4e7..7ae5f31 100644
--- a/pom.xml
+++ b/pom.xml
@@ -682,10 +682,10 @@
<module>tez-examples</module>
<module>tez-tests</module>
<module>tez-dag</module>
+ <module>tez-ext-service-tests</module>
<module>tez-ui</module>
<module>tez-plugins</module>
<module>tez-tools</module>
- <module>tez-ext-service-tests</module>
<module>tez-dist</module>
<module>docs</module>
</modules>
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
index 8337dcb..7f58f77 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
@@ -16,7 +16,6 @@ package org.apache.tez.serviceplugins.api;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.service.AbstractService;
import org.apache.tez.common.ServicePluginLifecycle;
/**
@@ -30,9 +29,6 @@ public abstract class ContainerLauncher implements ServicePluginLifecycle {
private final ContainerLauncherContext containerLauncherContext;
- // TODO TEZ-2003 Simplify this by moving away from AbstractService. Potentially Guava AbstractService.
- // A serviceInit(Configuration) is not likely to be very useful, and will expose unnecessary internal
- // configuration to the services if populated with the AM Configuration
public ContainerLauncher(ContainerLauncherContext containerLauncherContext) {
this.containerLauncherContext = containerLauncherContext;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
index a5b054f..9ff2bd5 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
@@ -27,6 +27,11 @@ import org.apache.tez.common.ServicePluginLifecycle;
@InterfaceStability.Unstable
public abstract class TaskScheduler implements ServicePluginLifecycle {
+ // TODO TEZ-2003 (post) TEZ-2668
+ // - Should setRegister / unregister be part of APIs when not YARN specific ?
+ // - Include vertex / task information in therequest so that the scheduler can make decisions
+ // around prioritizing tasks in the same vertex when others exist at the same priority.
+
private final TaskSchedulerContext taskSchedulerContext;
public TaskScheduler(TaskSchedulerContext taskSchedulerContext) {
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
index 6f37641..dbbf75c 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
@@ -53,7 +53,10 @@ public interface TaskSchedulerContext {
IDLE, RUNNING_APP, COMPLETED
}
- // TODO Post TEZ-2003. Remove references to YARN constructs like Container, ContainerStatus, NodeReport
+ // TODO TEZ-2003 (post) TEZ-2664. Remove references to YARN constructs like Container, ContainerStatus, NodeReport
+ // TODO TEZ-2003 (post) TEZ-2668 Enhancements to TaskScheduler interfaces
+ // - setApplicationRegistrationData may not be relevant to non YARN clusters
+ // - getAppFinalStatus may not be relevant to non YARN clusters
// upcall to app must be outside locks
public void taskAllocated(Object task,
Object appCookie,
@@ -78,7 +81,6 @@ public interface TaskSchedulerContext {
public float getProgress();
public void preemptContainer(ContainerId containerId);
- // TODO Post TEZ-2003. Another method which is primarily relevant to YARN clusters for unregistration.
public AppFinalStatus getFinalAppStatus();
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index f221414..794d390 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -27,9 +27,18 @@ import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.api.impl.TaskSpec;
-// TODO TEZ-2003 Move this into the tez-api module
+// TODO TEZ-2003 (post) TEZ-2665. Move to the tez-api module
+// TODO TEZ-2003 (post) TEZ-2664. Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
public abstract class TaskCommunicator implements ServicePluginLifecycle {
+ // TODO TEZ-2003 (post) TEZ-2666 Enhancements to interface
+ // - registerContainerEnd should provide the end reason / possible rename
+ // - get rid of getAddress
+ // - Add methods to support task preemption
+ // - Add a dagStarted notification, along with a payload
+ // - taskSpec breakup into a clean interface
+ // - Add methods to report task / container completion
+
private final TaskCommunicatorContext taskCommunicatorContext;
public TaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
@@ -52,36 +61,20 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
public void shutdown() throws Exception {
}
- // TODO Post TEZ-2003 Move this into the API module. Moving this requires abstractions for
- // TaskSpec and related classes. (assuming that's efficient for execution)
- // TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
- // TODO When talking to an external service, this plugin implementer may need access to a host:port
public abstract void registerRunningContainer(ContainerId containerId, String hostname, int port);
- // TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
public abstract void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason);
- // TODO TEZ-2003 Provide additional inforamtion like reason for container end / Task end.
- // Was it caused by preemption - or as a result of a general task completion / container completion
-
- // TODO TEZ-2003 TaskSpec breakup into a clean interface
- // TODO TEZ-2003 Add support for priority
public abstract void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
Map<String, LocalResource> additionalResources,
Credentials credentials,
boolean credentialsChanged, int priority);
- // TODO TEZ-2003. Are additional APIs required to mark a container as completed ? - for completeness.
-
- // TODO TEZ-2003 Remove reference to TaskAttemptID
public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID, TaskAttemptEndReason endReason);
- // TODO TEZ-2003 This doesn't necessarily belong here. A server may not start within the AM.
public abstract InetSocketAddress getAddress();
- // TODO Eventually. Add methods here to support preemption of tasks.
-
/**
* Receive notifications on vertex state changes.
* <p/>
@@ -108,8 +101,6 @@ public abstract class TaskCommunicator implements ServicePluginLifecycle {
* After this, the contents returned from querying the context may change at any point - due to
* the next dag being submitted.
*/
- // TODO TEZ-2003 This is extremely difficult to use. Add the dagStarted notification, and potentially
- // throw exceptions between a dagComplete and dagStart invocation.
public abstract void dagComplete(String dagName);
/**
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index a1e94a3..8073f6a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -28,23 +28,27 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
// Do not make calls into this from within a held lock.
-// TODO TEZ-2003 Move this into the tez-api module
+// TODO TEZ-2003 (post) TEZ-2665. Move to the tez-api module
public interface TaskCommunicatorContext {
- // TODO TEZ-2003 Add signalling back into this to indicate errors - e.g. Container unregsitered, task no longer running, etc.
-
- // TODO TEZ-2003 Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
+ // TODO TEZ-2003 (post) TEZ-2666 Enhancements to API
+ // - Consolidate usage of IDs
+ // - Split the heartbeat API to a liveness check and a status update
+ // - Rename and consolidate TaskHeartbeatResponse and TaskHeartbeatRequest
+ // - Fix taskStarted needs to be invoked before launching the actual task.
+ // - Potentially add methods to report availability stats to the scheduler
+ // - Report taskSuccess via a method instead of the heartbeat
+ // - Add methods to signal container / task state changes
+ // - Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
+ // - Handling of containres / tasks which no longer exist in the system (formalized interface instead of a shouldDie notification)
UserPayload getInitialUserPayload();
ApplicationAttemptId getApplicationAttemptId();
Credentials getCredentials();
- // TODO TEZ-2003 Move to vertex, taskIndex, version
boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
- // TODO TEZ-2003 Split the heartbeat API to a liveness check and a status update
- // KKK Rename this API
TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
boolean isKnownContainer(ContainerId containerId);
@@ -53,13 +57,10 @@ public interface TaskCommunicatorContext {
void containerAlive(ContainerId containerId);
- // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
void taskStartedRemotely(TezTaskAttemptID taskAttemptId, ContainerId containerId);
- // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
- // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
/**
@@ -72,9 +73,6 @@ public interface TaskCommunicatorContext {
* @param stateSet the set of states for which notifications are required. null implies all
*/
void registerForVertexStateUpdates(String vertexName, @Nullable Set<VertexState> stateSet);
- // TODO TEZ-2003 API. Should a method exist for task succeeded.
-
- // TODO Eventually Add methods to report availability stats to the scheduler.
/**
* Get the name of the currently executing dag
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
index b5ff991..d0c22d3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
@@ -19,13 +19,11 @@ import java.util.List;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.api.impl.TezEvent;
-// TODO TEZ-2003 Move this into the tez-api module
+// TODO TEZ-2003 (post) TEZ-2665. Move to the tez-api module
public class TaskHeartbeatRequest {
- // TODO TEZ-2003 Ideally containerIdentifier should not be part of the request.
- // Replace with a task lookup - vertex name + task index
+ // TODO TEZ-2003 (post) TEZ-2666 Ideally containerIdentifier should not be part of the request.
private final String containerIdentifier;
- // TODO TEZ-2003 Get rid of the task attemptId reference if possible
private final TezTaskAttemptID taskAttemptId;
private final List<TezEvent> events;
private final int startIndex;
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
index 7f063c4..dcf89ff 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
@@ -18,7 +18,7 @@ import java.util.List;
import org.apache.tez.runtime.api.impl.TezEvent;
-// TODO TEZ-2003 Move this into the tez-api module
+// TODO TEZ-2003 (post) TEZ-2665. Move to the tez-api module
public class TaskHeartbeatResponse {
private final boolean shouldDie;
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index cc109a6..941e583 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -162,7 +162,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
LOG.info("Using Default Local Task Communicator");
return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
} else {
- // TODO TEZ-2003. Use the payload
LOG.info("Using TaskCommunicator {}:{} " + taskCommDescriptor.getEntityName(), taskCommDescriptor.getClassName());
Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
.getClazz(taskCommDescriptor.getClassName());
@@ -217,7 +216,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
// This can happen when a task heartbeats. Meanwhile the container is unregistered.
// The information will eventually make it through to the plugin via a corresponding unregister.
// There's a race in that case between the unregister making it through, and this method returning.
- // TODO TEZ-2003. An exception back is likely a better approach than sending a shouldDie = true,
+ // TODO TEZ-2003 (post) TEZ-2666. An exception back is likely a better approach than sending a shouldDie = true,
// so that the plugin can handle the scenario. Alternately augment the response with error codes.
// Error codes would be better than exceptions.
LOG.info("Attempt: " + taskAttemptID + " is not recognized for heartbeats");
@@ -278,7 +277,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
String diagnostics) {
// Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
// and messages from the scheduler will release the container.
- // TODO TEZ-2003 Maybe consider un-registering here itself, since the task is not active anymore,
+ // TODO TEZ-2003 (post) TEZ-2671 Maybe consider un-registering here itself, since the task is not active anymore,
// instead of waiting for the unregister to flow through the Container.
// Fix along the same lines as TEZ-2124 by introducing an explict context.
context.getEventHandler().handle(new TaskAttemptEventAttemptKilled(taskAttemptId,
@@ -290,7 +289,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
String diagnostics) {
// Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
// and messages from the scheduler will release the container.
- // TODO TEZ-2003 Maybe consider un-registering here itself, since the task is not active anymore,
+ // TODO TEZ-2003 (post) TEZ-2671 Maybe consider un-registering here itself, since the task is not active anymore,
// instead of waiting for the unregister to flow through the Container.
// Fix along the same lines as TEZ-2124 by introducing an explict context.
context.getEventHandler().handle(new TaskAttemptEventAttemptFailed(taskAttemptId,
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index cc315b7..0f10305 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -43,6 +43,7 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
@InterfaceAudience.Private
public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, VertexStateUpdateListener {
+ // TODO TEZ-2003 (post) TEZ-2669 Propagate errors baack to the AM with proper error reporting
private final AppContext context;
private final TaskAttemptListenerImpTezDag taskAttemptListener;
@@ -188,7 +189,6 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
try {
taskAttemptListener.vertexStateUpdateNotificationReceived(event, taskCommunicatorIndex);
} catch (Exception e) {
- // TODO TEZ-2003 This needs to be propagated to the DAG as a user error.
throw new TezUncheckedException(e);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 2a5c80e..fb6d5e7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -463,9 +463,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
// Holder for Task information, which eventually will likely be VertexImplm taskIndex, attemptIndex
+ // TODO TEZ-2003. TEZ-2670. Remove this class.
protected static class TaskAttempt {
- // TODO TEZ-2003 Change this to work with VertexName, int id, int version
- // TODO TEZ-2003 Avoid constructing this unit all over the place
private TezTaskAttemptID taskAttemptId;
TaskAttempt(TezTaskAttemptID taskAttemptId) {
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index 25518b0..17f5675 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -1453,7 +1453,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
// check task resources, only check it in non-local mode
if (!appContext.isLocal()) {
for (Vertex v : vertexMap.values()) {
- // TODO TEZ-2003 (post) Ideally, this should be per source.
+ // TODO TEZ-2003 (post) TEZ-2624 Ideally, this should be per source.
if (v.getTaskResource().compareTo(appContext.getClusterInfo().getMaxContainerCapability()) > 0) {
String msg = "Vertex's TaskResource is beyond the cluster container capability," +
"Vertex=" + v.getLogIdentifier() +", Requested TaskResource=" + v.getTaskResource()
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
index 5085691..6520b44 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -189,7 +189,7 @@ public class TaskAttemptImpl implements TaskAttempt,
private final StateMachine<TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent> stateMachine;
- // TODO TEZ-2003 We may need some additional state management for STATUS_UPDATES, FAILED, KILLED coming in before
+ // TODO TEZ-2003 (post) TEZ-2667 We may need some additional state management for STATUS_UPDATES, FAILED, KILLED coming in before
// TASK_STARTED_REMOTELY. In case of a PUSH it's more intuitive to send TASK_STARTED_REMOTELY after communicating
// with the listening service and getting a response, which in turn can trigger STATUS_UPDATES / FAILED / KILLED
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 4c2e631..c86f638 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -502,7 +502,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
taskSchedulerServiceWrappers[i].start();
if (shouldUnregisterFlag.get()) {
// Flag may have been set earlier when task scheduler was not initialized
- // TODO TEZ-2003 Should setRegister / unregister be part of APIs when not YARN specific ?
// External services could need to talk to some other entity.
taskSchedulers[i].setShouldUnregister();
}
@@ -564,8 +563,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
appCallbackExecutor.awaitTermination(1000l, TimeUnit.MILLISECONDS);
}
- // TODO TEZ-2003 Consolidate TaskSchedulerAppCallback methods once these methods are moved into context
-
// TaskSchedulerAppCallback methods with schedulerId, where relevant
public synchronized void taskAllocated(int schedulerId, Object task,
Object appCookie,
@@ -651,7 +648,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
Resource maxContainerCapability,
Map<ApplicationAccessType, String> appAcls,
ByteBuffer clientAMSecretKey) {
- // TODO TEZ-2003 (post) Ideally clusterInfo should be available per source rather than a global view.
this.appContext.getClusterInfo().setMaxContainerCapability(
maxContainerCapability);
this.appAcls = appAcls;
@@ -751,7 +747,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
this.shouldUnregisterFlag.set(true);
for (int i = 0 ; i < taskSchedulers.length ; i++) {
if (this.taskSchedulers[i] != null) {
- // TODO TEZ-2003 registration required for all schedulers ?
this.taskSchedulers[i].setShouldUnregister();
}
}
@@ -764,7 +759,6 @@ public class TaskSchedulerEventHandler extends AbstractService implements
public boolean hasUnregistered() {
boolean result = true;
for (int i = 0 ; i < taskSchedulers.length ; i++) {
- // TODO TEZ-2003 registration required for all schedulers ?
result |= this.taskSchedulers[i].hasUnregistered();
if (result == false) {
return result;
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
index 0398882..682cd02 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
@@ -27,8 +27,6 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
public class AMContainerEventAssignTA extends AMContainerEvent {
- // TODO TEZ-2003. Add the task priority to this event.
-
private final TezTaskAttemptID attemptId;
// TODO Maybe have tht TAL pull the remoteTask from the TaskAttempt itself ?
private final TaskSpec remoteTaskSpec;
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index 907e129..f95f4ca 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -23,7 +23,6 @@
<version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
- <!-- TODO TEZ-2003 Merge this into the tez-tests module -->
<artifactId>tez-ext-service-tests</artifactId>
<dependencies>
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index f31a07b..845a27b 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -37,8 +37,6 @@ import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainer
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-// TODO TEZ-2003 look for all LOG.*(DBG and LOG.*(DEBUG messages
-
public class TezTestServiceContainerLauncher extends ContainerLauncher {
// TODO Support interruptability of tasks which haven't yet been launched.
@@ -119,7 +117,7 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
@Override
public void stopContainer(ContainerStopRequest stopRequest) {
- LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + stopRequest);
+ LOG.info("Ignoring stopContainer for event: " + stopRequest);
// that the container is actually done (normally received from RM)
// TODO Sending this out for an un-launched container is invalid
getContext().containerStopRequested(stopRequest.getContainerId());
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
index 7b42296..d265736 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
@@ -39,7 +39,7 @@ public class TezTestServiceNoOpContainerLauncher extends ContainerLauncher {
@Override
public void stopContainer(ContainerStopRequest stopRequest) {
- LOG.info("DEBUG: Ignoring STOP_REQUEST {}", stopRequest);
+ LOG.info("Ignoring stopRequest {}", stopRequest);
getContext().containerStopRequested(stopRequest.getContainerId());
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 0d87995..17f8a87 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -151,12 +151,12 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
@Override
public void blacklistNode(NodeId nodeId) {
- LOG.info("DEBUG: BlacklistNode not supported");
+ LOG.info("BlacklistNode not supported");
}
@Override
public void unblacklistNode(NodeId nodeId) {
- LOG.info("DEBUG: unBlacklistNode not supported");
+ LOG.info("unBlacklistNode not supported");
}
@Override
@@ -195,7 +195,7 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
@Override
public Object deallocateContainer(ContainerId containerId) {
- LOG.info("DEBUG: Ignoring deallocateContainer for containerId: " + containerId);
+ LOG.info("Ignoring deallocateContainer for containerId: " + containerId);
return null;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index f3fc442..472a43c 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -173,7 +173,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
throw new TezException(e);
}
}
- LOG.info("DEBUG: Dirs are: " + Arrays.toString(localDirs));
+ LOG.info("Dirs for {} are {}", request.getContainerIdString(), Arrays.toString(localDirs));
// Setup workingDir. This is otherwise setup as Environment.PWD
@@ -193,7 +193,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
// TODO Unregistering does not happen at the moment, since there's no signals on when an app completes.
- LOG.info("DEBUG: Registering request with the ShuffleHandler");
+ LOG.info("Registering request with the ShuffleHandler for containerId {}", request.getContainerIdString());
ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser());
@@ -255,7 +255,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
// TODO Unregistering does not happen at the moment, since there's no signals on when an app completes.
- LOG.info("DEBUG: Registering request with the ShuffleHandler");
+ LOG.info("Registering request with the ShuffleHandler for containerId {}", request.getContainerIdString());
ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser());
TaskRunnerCallable callable = new TaskRunnerCallable(request, new Configuration(getConfig()),
new ExecutionContextImpl(localAddress.get().getHostName()), env, localDirs,
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 2c52ae3..3701455 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -130,8 +130,6 @@ public class TestExternalTezServices {
confForJobs.set(entry.getKey(), entry.getValue());
}
- // TODO TEZ-2003 Once per vertex configuration is possible, run separate tests for push vs pull (regular threaded execution)
-
Path stagingDirPath = new Path("/tmp/tez-staging-dir");
remoteFs.mkdirs(stagingDirPath);
// This is currently configured to push tasks into the Service, and then use the standard RPC
http://git-wip-us.apache.org/repos/asf/tez/blob/4766b71c/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
index 47a61ab..9a5a3ab 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
@@ -26,8 +26,6 @@ import org.apache.tez.runtime.task.ErrorReporter;
public interface TaskReporterInterface {
- // TODO TEZ-2003 Consolidate private API usage if making this public
-
void registerTask(RuntimeTask task, ErrorReporter errorReporter);
void unregisterTask(TezTaskAttemptID taskAttemptId);
@@ -43,4 +41,4 @@ public interface TaskReporterInterface {
void shutdown();
-}
+}
\ No newline at end of file
[34/51] [abbrv] tez git commit: TEZ-2621. rebase 07/14. (sseth)
Posted by ss...@apache.org.
TEZ-2621. rebase 07/14. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/0539d315
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/0539d315
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/0539d315
Branch: refs/heads/TEZ-2003
Commit: 0539d315d1ef716267e3ce81c188e404133a47e2
Parents: b5459e2
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jul 14 17:26:25 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../app/TestTaskAttemptListenerImplTezDag.java | 25 +++++++++++++++++---
.../tez/dag/app/dag/impl/TestTaskAttempt.java | 7 ++++--
3 files changed, 28 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/0539d315/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 1e8abcf..590fe7f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -32,5 +32,6 @@ ALL CHANGES:
TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
TEZ-2508. rebase 06/01
TEZ-2526. Fix version for tez-history-parser.
+ TEZ-2621. rebase 07/14
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/0539d315/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 68d3baf..7f0362d 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -35,7 +35,9 @@ import java.util.Map;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -45,7 +47,9 @@ import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ContainerContext;
import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.common.security.TokenCache;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.ContainerEndReason;
import org.apache.tez.dag.api.TaskAttemptEndReason;
@@ -284,11 +288,18 @@ public class TestTaskAttemptListenerImplTezDag {
}
}
+ // TODO TEZ-2003 Move this into TestTezTaskCommunicator. Potentially other tests as well.
@Test (timeout= 5000)
public void testPortRange_NotSpecified() {
Configuration conf = new Configuration();
+ JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
+ "fakeIdentifier"));
+ Token<JobTokenIdentifier> sessionToken = new Token<JobTokenIdentifier>(identifier,
+ new JobTokenSecretManager());
+ sessionToken.setService(identifier.getJobId());
+ TokenCache.setSessionToken(sessionToken, credentials);
taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
// no exception happen, should started properly
taskAttemptListener.init(conf);
taskAttemptListener.start();
@@ -298,12 +309,20 @@ public class TestTaskAttemptListenerImplTezDag {
boolean succeedToAllocate = true;
try {
Configuration conf = new Configuration();
+
+ JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
+ "fakeIdentifier"));
+ Token<JobTokenIdentifier> sessionToken = new Token<JobTokenIdentifier>(identifier,
+ new JobTokenSecretManager());
+ sessionToken.setService(identifier.getJobId());
+ TokenCache.setSessionToken(sessionToken, credentials);
+
conf.set(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE, port + "-" + port);
taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
taskAttemptListener.init(conf);
taskAttemptListener.start();
- int resultedPort = taskAttemptListener.getAddress().getPort();
+ int resultedPort = taskAttemptListener.getTaskCommunicator(0).getAddress().getPort();
assertEquals(port, resultedPort);
} catch (Exception e) {
succeedToAllocate = false;
http://git-wip-us.apache.org/repos/asf/tez/blob/0539d315/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
index d6fc46e..2bf1c85 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
@@ -21,6 +21,7 @@ package org.apache.tez.dag.app.dag.impl;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
@@ -1270,8 +1271,10 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = spy(new MockEventHandler());
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
+ TaskCommunicator mockTaskComm = mock(TaskCommunicator.class);
+ when(mockTaskComm.getAddress()).thenReturn(
new InetSocketAddress("localhost", 0));
+ when(taListener.getTaskCommunicator(any(int.class))).thenReturn(mockTaskComm);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1292,7 +1295,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
[28/51] [abbrv] tez git commit: TEZ-2438. tez-tools version in the
branch is incorrect. (sseth)
Posted by ss...@apache.org.
TEZ-2438. tez-tools version in the branch is incorrect. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/d8fb6adc
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/d8fb6adc
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/d8fb6adc
Branch: refs/heads/TEZ-2003
Commit: d8fb6adca73abe71b0d31bc7f11720577c48910f
Parents: 960d3b9
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon May 11 16:42:45 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
tez-tools/pom.xml | 2 +-
tez-tools/tez-tfile-parser/pom.xml | 2 +-
3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/d8fb6adc/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index ad167ab..66c110f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -23,5 +23,6 @@ ALL CHANGES:
TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups.
TEZ-2420. TaskRunner returning before executing the task.
TEZ-2433. Fixes after rebase 05/08
+ TEZ-2438. tez-tools version in the branch is incorrect.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/d8fb6adc/tez-tools/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml
index ed13143..5b9e41a 100644
--- a/tez-tools/pom.xml
+++ b/tez-tools/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-tools</artifactId>
<packaging>pom</packaging>
http://git-wip-us.apache.org/repos/asf/tez/blob/d8fb6adc/tez-tools/tez-tfile-parser/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml
index aea1299..9467bac 100644
--- a/tez-tools/tez-tfile-parser/pom.xml
+++ b/tez-tools/tez-tfile-parser/pom.xml
@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez-tools</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-tfile-parser</artifactId>
[25/51] [abbrv] tez git commit: TEZ-2433. Fixes after rebase 05/08.
(sseth)
Posted by ss...@apache.org.
TEZ-2433. Fixes after rebase 05/08. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/960d3b9c
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/960d3b9c
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/960d3b9c
Branch: refs/heads/TEZ-2003
Commit: 960d3b9cfb07615bb44f7883cd09cb572cebff2f
Parents: 7f0b967
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 8 18:43:16 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../tez/dag/api/TaskHeartbeatResponse.java | 10 ++++++--
.../dag/app/TaskAttemptListenerImpTezDag.java | 27 ++++++++++----------
.../tez/dag/app/TezTaskCommunicatorImpl.java | 9 +++----
.../app/TestTaskAttemptListenerImplTezDag.java | 10 +++-----
.../library/common/shuffle/TestFetcher.java | 8 ++----
6 files changed, 31 insertions(+), 34 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/960d3b9c/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9b2339f..ad167ab 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -22,5 +22,6 @@ ALL CHANGES:
TEZ-2388. Send dag identifier as part of the fetcher request string.
TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups.
TEZ-2420. TaskRunner returning before executing the task.
+ TEZ-2433. Fixes after rebase 05/08
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/960d3b9c/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
index c82a743..b826e76 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
@@ -22,11 +22,13 @@ import org.apache.tez.runtime.api.impl.TezEvent;
public class TaskHeartbeatResponse {
private final boolean shouldDie;
- private List<TezEvent> events;
+ private final int nextFromEventId;
+ private final List<TezEvent> events;
- public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events) {
+ public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events, int nextFromEventId) {
this.shouldDie = shouldDie;
this.events = events;
+ this.nextFromEventId = nextFromEventId;
}
public boolean isShouldDie() {
@@ -36,4 +38,8 @@ public class TaskHeartbeatResponse {
public List<TezEvent> getEvents() {
return events;
}
+
+ public int getNextFromEventId() {
+ return nextFromEventId;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/960d3b9c/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index cbaed99..db78fa9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -78,7 +78,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
protected final TaskHeartbeatHandler taskHeartbeatHandler;
protected final ContainerHeartbeatHandler containerHeartbeatHandler;
- private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null);
+ private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null, 0);
private final ConcurrentMap<TezTaskAttemptID, ContainerId> registeredAttempts =
new ConcurrentHashMap<TezTaskAttemptID, ContainerId>();
@@ -194,7 +194,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
// So - avoiding synchronization.
pingContainerHeartbeatHandler(containerId);
- List<TezEvent> outEvents = null;
+ TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(0, null);
TezTaskAttemptID taskAttemptID = request.getTaskAttemptId();
if (taskAttemptID != null) {
ContainerId containerIdFromMap = registeredAttempts.get(taskAttemptID);
@@ -216,12 +216,17 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
List<TezEvent> otherEvents = new ArrayList<TezEvent>();
+ // route TASK_STATUS_UPDATE_EVENT directly to TaskAttempt and route other events
+ // (DATA_MOVEMENT_EVENT, TASK_ATTEMPT_COMPLETED_EVENT, TASK_ATTEMPT_FAILED_EVENT)
+ // to VertexImpl to ensure the events ordering
+ // 1. DataMovementEvent is logged as RecoveryEvent before TaskAttemptFinishedEvent
+ // 2. TaskStatusEvent is handled before TaskAttemptFinishedEvent
for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
final EventType eventType = tezEvent.getEventType();
- if (eventType == EventType.TASK_STATUS_UPDATE_EVENT ||
- eventType == EventType.TASK_ATTEMPT_COMPLETED_EVENT) {
- context.getEventHandler()
- .handle(getTaskAttemptEventFromTezEvent(taskAttemptID, tezEvent));
+ if (eventType == EventType.TASK_STATUS_UPDATE_EVENT) {
+ TaskAttemptEvent taskAttemptEvent = new TaskAttemptEventStatusUpdate(taskAttemptID,
+ (TaskStatusUpdateEvent) tezEvent.getEvent());
+ context.getEventHandler().handle(taskAttemptEvent);
} else {
otherEvents.add(tezEvent);
}
@@ -232,14 +237,13 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(otherEvents)));
}
taskHeartbeatHandler.pinged(taskAttemptID);
- outEvents = context
+ eventInfo = context
.getCurrentDAG()
.getVertex(taskAttemptID.getTaskID().getVertexID())
- .getTask(taskAttemptID.getTaskID())
.getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(),
request.getMaxEvents());
}
- return new TaskHeartbeatResponse(false, outEvents);
+ return new TaskHeartbeatResponse(false, eventInfo.getEvents(), eventInfo.getNextFromEventId());
}
public void taskAlive(TezTaskAttemptID taskAttemptId) {
taskHeartbeatHandler.pinged(taskAttemptId);
@@ -435,9 +439,4 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
+ ", ContainerId not known for this attempt");
}
}
-
-
- public TaskCommunicator getTaskCommunicator() {
- return taskCommunicators[0];
- }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/960d3b9c/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 6200a5b..accde2c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -364,13 +364,10 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
request.getMaxEvents());
tResponse = taskCommunicatorContext.heartbeat(tRequest);
}
- TezHeartbeatResponse response;
- if (tResponse == null) {
- response = new TezHeartbeatResponse();
- } else {
- response = new TezHeartbeatResponse(tResponse.getEvents());
- }
+ TezHeartbeatResponse response = new TezHeartbeatResponse();
response.setLastRequestId(requestId);
+ response.setEvents(tResponse.getEvents());
+ response.setNextFromEventId(tResponse.getNextFromEventId());
containerInfo.lastRequestId = requestId;
containerInfo.lastResponse = response;
return response;
http://git-wip-us.apache.org/repos/asf/tez/blob/960d3b9c/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 2208220..34b9792 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -48,6 +48,7 @@ import org.apache.tez.common.ContainerTask;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezException;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.tez.common.TezTaskUmbilicalProtocol;
@@ -70,8 +71,6 @@ import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
import org.apache.tez.runtime.api.impl.EventType;
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.api.impl.TezEvent;
-import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
-import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
@@ -260,10 +259,9 @@ public class TestTaskAttemptListenerImplTezDag {
public void testTaskHeartbeatResponse() throws Exception {
List<TezEvent> events = new ArrayList<TezEvent>();
List<TezEvent> eventsToSend = new ArrayList<TezEvent>();
- TezHeartbeatResponse response = generateHeartbeat(events, 0, 1, 2, eventsToSend);
+ TaskHeartbeatResponse response = generateHeartbeat(events, 0, 1, 2, eventsToSend);
assertEquals(2, response.getNextFromEventId());
- assertEquals(1, response.getLastRequestId());
assertEquals(eventsToSend, response.getEvents());
}
@@ -320,7 +318,7 @@ public class TestTaskAttemptListenerImplTezDag {
return succeedToAllocate;
}
- private TezHeartbeatResponse generateHeartbeat(List<TezEvent> events,
+ private TaskHeartbeatResponse generateHeartbeat(List<TezEvent> events,
int fromEventId, int maxEvents, int nextFromEventId,
List<TezEvent> sendEvents) throws IOException, TezException {
ContainerId containerId = createContainerId(appId, 1);
@@ -335,7 +333,7 @@ public class TestTaskAttemptListenerImplTezDag {
taskAttemptListener.registerTaskAttempt(amContainerTask, containerId, 0);
TaskHeartbeatRequest request = mock(TaskHeartbeatRequest.class);
-
+ doReturn(containerId.toString()).when(request).getContainerIdentifier();
doReturn(containerId.toString()).when(request).getContainerIdentifier();
doReturn(taskAttemptID).when(request).getTaskAttemptId();
doReturn(events).when(request).getEvents();
http://git-wip-us.apache.org/repos/asf/tez/blob/960d3b9c/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
index 85e3540..08efb3e 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
@@ -31,7 +31,6 @@ import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import java.io.IOException;
-import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Iterator;
import java.util.LinkedList;
@@ -43,11 +42,8 @@ import com.google.common.collect.Lists;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.runtime.api.ExecutionContext;
-import org.apache.tez.runtime.api.InputContext;
import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
import org.apache.tez.runtime.library.common.InputIdentifier;
@@ -93,7 +89,7 @@ public class TestFetcher {
// when enabled and hostname does not match use http fetch.
builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
- ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
+ ApplicationId.newInstance(0, 1), -1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
PORT, false);
builder.assignWork(HOST + "_OTHER", PORT, 0, Arrays.asList(srcAttempts));
fetcher = spy(builder.build());
@@ -109,7 +105,7 @@ public class TestFetcher {
// when enabled and port does not match use http fetch.
builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
- ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, PORT, false);
+ ApplicationId.newInstance(0, 1), -1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, PORT, false);
builder.assignWork(HOST, PORT + 1, 0, Arrays.asList(srcAttempts));
fetcher = spy(builder.build());
[02/51] [abbrv] tez git commit: TEZ-2090. Add tests for jobs running
in external services. (sseth)
Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
new file mode 100644
index 0000000..4a6ce33
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -0,0 +1,512 @@
+/*
+ * 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.tez.service.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
+import org.apache.log4j.Logger;
+import org.apache.tez.common.TezCommonUtils;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.runtime.task.TaskReporter;
+import org.apache.tez.runtime.task.TezTaskRunner;
+import org.apache.tez.service.ContainerRunner;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.runtime.api.ExecutionContext;
+import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
+import org.apache.tez.runtime.task.TezChild;
+import org.apache.tez.runtime.task.TezChild.ContainerExecutionResult;
+import org.apache.tez.shufflehandler.ShuffleHandler;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.util.ProtoConverters;
+
+public class ContainerRunnerImpl extends AbstractService implements ContainerRunner {
+
+ private static final Logger LOG = Logger.getLogger(ContainerRunnerImpl.class);
+
+ private final ListeningExecutorService executorService;
+ private final AtomicReference<InetSocketAddress> localAddress;
+ private final String[] localDirsBase;
+ private final Map<String, String> localEnv = new HashMap<String, String>();
+ private volatile FileSystem localFs;
+ private final long memoryPerExecutor;
+ // TODO Support for removing queued containers, interrupting / killing specific containers - when preemption is supported
+
+
+
+
+ public ContainerRunnerImpl(int numExecutors, String[] localDirsBase,
+ AtomicReference<InetSocketAddress> localAddress,
+ long totalMemoryAvailableBytes) {
+ super("ContainerRunnerImpl");
+ Preconditions.checkState(numExecutors > 0,
+ "Invalid number of executors: " + numExecutors + ". Must be > 0");
+ this.localDirsBase = localDirsBase;
+ this.localAddress = localAddress;
+
+ ExecutorService raw = Executors.newFixedThreadPool(numExecutors,
+ new ThreadFactoryBuilder().setNameFormat("ContainerExecutor %d").build());
+ this.executorService = MoreExecutors.listeningDecorator(raw);
+
+
+ // 80% of memory considered for accounted buffers. Rest for objects.
+ // TODO Tune this based on the available size.
+ this.memoryPerExecutor = (long)(totalMemoryAvailableBytes * 0.8 / (float) numExecutors);
+
+ LOG.info("ContainerRunnerImpl config: " +
+ "memoryPerExecutorDerived=" + memoryPerExecutor +
+ ", numExecutors=" + numExecutors
+ );
+ }
+
+ @Override
+ public void serviceInit(Configuration conf) {
+ try {
+ localFs = FileSystem.getLocal(conf);
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to setup local filesystem instance", e);
+ }
+ }
+
+ @Override
+ public void serviceStart() {
+ }
+
+ public void setShufflePort(int shufflePort) {
+ AuxiliaryServiceHelper.setServiceDataIntoEnv(
+ TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
+ ByteBuffer.allocate(4).putInt(shufflePort), localEnv);
+ }
+
+ @Override
+ protected void serviceStop() throws Exception {
+ super.serviceStop();
+ }
+
+ // TODO Move this into a utilities class
+ private static String createAppSpecificLocalDir(String baseDir, String applicationIdString,
+ String user) {
+ return baseDir + File.separator + "usercache" + File.separator + user + File.separator +
+ "appcache" + File.separator + applicationIdString;
+ }
+
+ /**
+ * Submit a container which is ready for running.
+ * The regular pull mechanism will be used to fetch work from the AM
+ * @param request
+ * @throws IOException
+ */
+ @Override
+ public void queueContainer(RunContainerRequestProto request) throws IOException {
+ LOG.info("Queuing container for execution: " + request);
+
+ Map<String, String> env = new HashMap<String, String>();
+ env.putAll(localEnv);
+ env.put(ApplicationConstants.Environment.USER.name(), request.getUser());
+
+ String[] localDirs = new String[localDirsBase.length];
+
+ // Setup up local dirs to be application specific, and create them.
+ for (int i = 0; i < localDirsBase.length; i++) {
+ localDirs[i] = createAppSpecificLocalDir(localDirsBase[i], request.getApplicationIdString(),
+ request.getUser());
+ localFs.mkdirs(new Path(localDirs[i]));
+ }
+ LOG.info("DEBUG: Dirs are: " + Arrays.toString(localDirs));
+
+
+ // Setup workingDir. This is otherwise setup as Environment.PWD
+ // Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
+ String workingDir = localDirs[0];
+
+ Credentials credentials = new Credentials();
+ DataInputBuffer dib = new DataInputBuffer();
+ byte[] tokenBytes = request.getCredentialsBinary().toByteArray();
+ dib.reset(tokenBytes, tokenBytes.length);
+ credentials.readTokenStorageStream(dib);
+
+ Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
+
+ // TODO Unregistering does not happen at the moment, since there's no signals on when an app completes.
+ LOG.info("DEBUG: Registering request with the ShuffleHandler");
+ ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser());
+
+
+ ContainerRunnerCallable callable = new ContainerRunnerCallable(request, new Configuration(getConfig()),
+ new ExecutionContextImpl(localAddress.get().getHostName()), env, localDirs,
+ workingDir, credentials, memoryPerExecutor);
+ ListenableFuture<ContainerExecutionResult> future = executorService
+ .submit(callable);
+ Futures.addCallback(future, new ContainerRunnerCallback(request, callable));
+ }
+
+ /**
+ * Submit an entire work unit - containerId + TaskSpec.
+ * This is intended for a task push from the AM
+ *
+ * @param request
+ * @throws IOException
+ */
+ @Override
+ public void submitWork(SubmitWorkRequestProto request) throws
+ IOException {
+ LOG.info("Queuing work for execution: " + request);
+
+ Map<String, String> env = new HashMap<String, String>();
+ env.putAll(localEnv);
+ env.put(ApplicationConstants.Environment.USER.name(), request.getUser());
+
+ String[] localDirs = new String[localDirsBase.length];
+
+ // Setup up local dirs to be application specific, and create them.
+ for (int i = 0; i < localDirsBase.length; i++) {
+ localDirs[i] = createAppSpecificLocalDir(localDirsBase[i], request.getApplicationIdString(),
+ request.getUser());
+ localFs.mkdirs(new Path(localDirs[i]));
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Dirs are: " + Arrays.toString(localDirs));
+ }
+
+ // Setup workingDir. This is otherwise setup as Environment.PWD
+ // Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
+ String workingDir = localDirs[0];
+
+ Credentials credentials = new Credentials();
+ DataInputBuffer dib = new DataInputBuffer();
+ byte[] tokenBytes = request.getCredentialsBinary().toByteArray();
+ dib.reset(tokenBytes, tokenBytes.length);
+ credentials.readTokenStorageStream(dib);
+
+ Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
+
+ // TODO Unregistering does not happen at the moment, since there's no signals on when an app completes.
+ LOG.info("DEBUG: Registering request with the ShuffleHandler");
+ ShuffleHandler.get().registerApplication(request.getApplicationIdString(), jobToken, request.getUser());
+ TaskRunnerCallable callable = new TaskRunnerCallable(request, new Configuration(getConfig()),
+ new ExecutionContextImpl(localAddress.get().getHostName()), env, localDirs,
+ workingDir, credentials, memoryPerExecutor);
+ ListenableFuture<ContainerExecutionResult> future = executorService.submit(callable);
+ Futures.addCallback(future, new TaskRunnerCallback(request, callable));
+ }
+
+
+ static class ContainerRunnerCallable implements Callable<ContainerExecutionResult> {
+
+ private final RunContainerRequestProto request;
+ private final Configuration conf;
+ private final String workingDir;
+ private final String[] localDirs;
+ private final Map<String, String> envMap;
+ private final String pid = null;
+ private final ObjectRegistryImpl objectRegistry;
+ private final ExecutionContext executionContext;
+ private final Credentials credentials;
+ private final long memoryAvailable;
+ private volatile TezChild tezChild;
+
+
+ ContainerRunnerCallable(RunContainerRequestProto request, Configuration conf,
+ ExecutionContext executionContext, Map<String, String> envMap,
+ String[] localDirs, String workingDir, Credentials credentials,
+ long memoryAvailable) {
+ this.request = request;
+ this.conf = conf;
+ this.executionContext = executionContext;
+ this.envMap = envMap;
+ this.workingDir = workingDir;
+ this.localDirs = localDirs;
+ this.objectRegistry = new ObjectRegistryImpl();
+ this.credentials = credentials;
+ this.memoryAvailable = memoryAvailable;
+
+ }
+
+ @Override
+ public ContainerExecutionResult call() throws Exception {
+ Stopwatch sw = new Stopwatch().start();
+ tezChild =
+ new TezChild(conf, request.getAmHost(), request.getAmPort(),
+ request.getContainerIdString(),
+ request.getTokenIdentifier(), request.getAppAttemptNumber(), workingDir, localDirs,
+ envMap, objectRegistry, pid,
+ executionContext, credentials, memoryAvailable, request.getUser());
+ ContainerExecutionResult result = tezChild.run();
+ LOG.info("ExecutionTime for Container: " + request.getContainerIdString() + "=" +
+ sw.stop().elapsedMillis());
+ return result;
+ }
+
+ public TezChild getTezChild() {
+ return this.tezChild;
+ }
+ }
+
+
+ final class ContainerRunnerCallback implements FutureCallback<ContainerExecutionResult> {
+
+ private final RunContainerRequestProto request;
+ private final ContainerRunnerCallable containerRunnerCallable;
+
+ ContainerRunnerCallback(RunContainerRequestProto request,
+ ContainerRunnerCallable containerRunnerCallable) {
+ this.request = request;
+ this.containerRunnerCallable = containerRunnerCallable;
+ }
+
+ // TODO Proper error handling
+ @Override
+ public void onSuccess(ContainerExecutionResult result) {
+ switch (result.getExitStatus()) {
+ case SUCCESS:
+ LOG.info("Successfully finished: " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString());
+ break;
+ case EXECUTION_FAILURE:
+ LOG.info("Failed to run: " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString(), result.getThrowable());
+ break;
+ case INTERRUPTED:
+ LOG.info(
+ "Interrupted while running: " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString(), result.getThrowable());
+ break;
+ case ASKED_TO_DIE:
+ LOG.info(
+ "Asked to die while running: " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString());
+ break;
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable t) {
+ LOG.error(
+ "TezChild execution failed for : " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString(), t);
+ TezChild tezChild = containerRunnerCallable.getTezChild();
+ if (tezChild != null) {
+ tezChild.shutdown();
+ }
+ }
+ }
+
+ static class TaskRunnerCallable implements Callable<ContainerExecutionResult> {
+
+ private final SubmitWorkRequestProto request;
+ private final Configuration conf;
+ private final String workingDir;
+ private final String[] localDirs;
+ private final Map<String, String> envMap;
+ private final String pid = null;
+ private final ObjectRegistryImpl objectRegistry;
+ private final ExecutionContext executionContext;
+ private final Credentials credentials;
+ private final long memoryAvailable;
+ private final ListeningExecutorService executor;
+ private volatile TezTaskRunner taskRunner;
+ private volatile TaskReporter taskReporter;
+ private TezTaskUmbilicalProtocol umbilical;
+
+
+ TaskRunnerCallable(SubmitWorkRequestProto request, Configuration conf,
+ ExecutionContext executionContext, Map<String, String> envMap,
+ String[] localDirs, String workingDir, Credentials credentials,
+ long memoryAvailable) {
+ this.request = request;
+ this.conf = conf;
+ this.executionContext = executionContext;
+ this.envMap = envMap;
+ this.workingDir = workingDir;
+ this.localDirs = localDirs;
+ this.objectRegistry = new ObjectRegistryImpl();
+ this.credentials = credentials;
+ this.memoryAvailable = memoryAvailable;
+ // TODO This executor seems unnecessary. Here and TezChild
+ ExecutorService executorReal = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder()
+ .setDaemon(true)
+ .setNameFormat("TezTaskRunner_" + request.getTaskSpec().getTaskAttemptIdString()).build());
+ executor = MoreExecutors.listeningDecorator(executorReal);
+ }
+
+ @Override
+ public ContainerExecutionResult call() throws Exception {
+
+ // TODO Consolidate this code with TezChild.
+ Stopwatch sw = new Stopwatch().start();
+ UserGroupInformation taskUgi = UserGroupInformation.createRemoteUser(request.getUser());
+ taskUgi.addCredentials(credentials);
+
+ Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
+ Map<String, ByteBuffer> serviceConsumerMetadata = new HashMap<String, ByteBuffer>();
+ serviceConsumerMetadata.put(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
+ TezCommonUtils.convertJobTokenToBytes(jobToken));
+ Multimap<String, String> startedInputsMap = HashMultimap.create();
+
+ UserGroupInformation taskOwner =
+ UserGroupInformation.createRemoteUser(request.getTokenIdentifier());
+ final InetSocketAddress address =
+ NetUtils.createSocketAddrForHost(request.getAmHost(), request.getAmPort());
+ SecurityUtil.setTokenService(jobToken, address);
+ taskOwner.addToken(jobToken);
+ umbilical = taskOwner.doAs(new PrivilegedExceptionAction<TezTaskUmbilicalProtocol>() {
+ @Override
+ public TezTaskUmbilicalProtocol run() throws Exception {
+ return RPC.getProxy(TezTaskUmbilicalProtocol.class,
+ TezTaskUmbilicalProtocol.versionID, address, conf);
+ }
+ });
+ // TODO Stop reading this on each request.
+ taskReporter = new TaskReporter(
+ umbilical,
+ conf.getInt(TezConfiguration.TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS,
+ TezConfiguration.TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS_DEFAULT),
+ conf.getLong(
+ TezConfiguration.TEZ_TASK_AM_HEARTBEAT_COUNTER_INTERVAL_MS,
+ TezConfiguration.TEZ_TASK_AM_HEARTBEAT_COUNTER_INTERVAL_MS_DEFAULT),
+ conf.getInt(TezConfiguration.TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT,
+ TezConfiguration.TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT_DEFAULT),
+ new AtomicLong(0),
+ request.getContainerIdString());
+
+ taskRunner = new TezTaskRunner(conf, taskUgi, localDirs,
+ ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()), umbilical,
+ request.getAppAttemptNumber(),
+ serviceConsumerMetadata, envMap, startedInputsMap, taskReporter, executor, objectRegistry,
+ pid,
+ executionContext, memoryAvailable);
+
+ boolean shouldDie;
+ try {
+ shouldDie = !taskRunner.run();
+ if (shouldDie) {
+ LOG.info("Got a shouldDie notification via hearbeats. Shutting down");
+ return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.SUCCESS, null,
+ "Asked to die by the AM");
+ }
+ } catch (IOException e) {
+ return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
+ e, "TaskExecutionFailure: " + e.getMessage());
+ } catch (TezException e) {
+ return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
+ e, "TaskExecutionFailure: " + e.getMessage());
+ } finally {
+ FileSystem.closeAllForUGI(taskUgi);
+ }
+ LOG.info("ExecutionTime for Container: " + request.getContainerIdString() + "=" +
+ sw.stop().elapsedMillis());
+ return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.SUCCESS, null,
+ null);
+ }
+
+ public void shutdown() {
+ executor.shutdownNow();
+ if (taskReporter != null) {
+ taskReporter.shutdown();
+ }
+ if (umbilical != null) {
+ RPC.stopProxy(umbilical);
+ }
+ }
+ }
+
+
+ final class TaskRunnerCallback implements FutureCallback<ContainerExecutionResult> {
+
+ private final SubmitWorkRequestProto request;
+ private final TaskRunnerCallable taskRunnerCallable;
+
+ TaskRunnerCallback(SubmitWorkRequestProto request,
+ TaskRunnerCallable containerRunnerCallable) {
+ this.request = request;
+ this.taskRunnerCallable = containerRunnerCallable;
+ }
+
+ // TODO Proper error handling
+ @Override
+ public void onSuccess(ContainerExecutionResult result) {
+ switch (result.getExitStatus()) {
+ case SUCCESS:
+ LOG.info("Successfully finished: " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString());
+ break;
+ case EXECUTION_FAILURE:
+ LOG.info("Failed to run: " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString(), result.getThrowable());
+ break;
+ case INTERRUPTED:
+ LOG.info(
+ "Interrupted while running: " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString(), result.getThrowable());
+ break;
+ case ASKED_TO_DIE:
+ LOG.info(
+ "Asked to die while running: " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString());
+ break;
+ }
+ taskRunnerCallable.shutdown();
+ }
+
+ @Override
+ public void onFailure(Throwable t) {
+ LOG.error(
+ "TezTaskRunner execution failed for : " + request.getApplicationIdString() + ", containerId=" +
+ request.getContainerIdString(), t);
+ taskRunnerCallable.shutdown();
+ }
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
new file mode 100644
index 0000000..012e352
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
@@ -0,0 +1,126 @@
+/*
+ * 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.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.tez.service.ContainerRunner;
+import org.apache.tez.shufflehandler.ShuffleHandler;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+
+public class TezTestService extends AbstractService implements ContainerRunner {
+
+ private static final Logger LOG = Logger.getLogger(TezTestService.class);
+
+ private final Configuration shuffleHandlerConf;
+ private final int numExecutors;
+
+ private final TezTestServiceProtocolServerImpl server;
+ private final ContainerRunnerImpl containerRunner;
+ private final String[] localDirs;
+
+ private final AtomicInteger numSubmissions = new AtomicInteger(0);
+
+
+ private final AtomicReference<InetSocketAddress> address = new AtomicReference<InetSocketAddress>();
+
+ public TezTestService(Configuration conf, int numExecutors, long memoryAvailable, String[] localDirs) {
+ super(TezTestService.class.getSimpleName());
+ this.numExecutors = numExecutors;
+ this.localDirs = localDirs;
+
+ long memoryAvailableBytes = memoryAvailable;
+ long jvmMax = Runtime.getRuntime().maxMemory();
+
+ LOG.info(TezTestService.class.getSimpleName() + " created with the following configuration: " +
+ "numExecutors=" + numExecutors +
+ ", workDirs=" + Arrays.toString(localDirs) +
+ ", memoryAvailable=" + memoryAvailable +
+ ", jvmMaxMemory=" + jvmMax);
+
+ Preconditions.checkArgument(this.numExecutors > 0);
+ Preconditions.checkArgument(this.localDirs != null && this.localDirs.length > 0,
+ "Work dirs must be specified");
+ Preconditions.checkState(jvmMax >= memoryAvailableBytes,
+ "Invalid configuration. Xmx value too small. maxAvailable=" + jvmMax + ", configured=" +
+ memoryAvailableBytes);
+
+ this.shuffleHandlerConf = new Configuration(conf);
+ // Start Shuffle on a random port
+ this.shuffleHandlerConf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+ this.shuffleHandlerConf.set(ShuffleHandler.SHUFFLE_HANDLER_LOCAL_DIRS, StringUtils.arrayToString(localDirs));
+
+ this.server = new TezTestServiceProtocolServerImpl(this, address);
+ this.containerRunner = new ContainerRunnerImpl(numExecutors, localDirs, address,
+ memoryAvailableBytes);
+ }
+
+ @Override
+ public void serviceInit(Configuration conf) {
+ server.init(conf);
+ containerRunner.init(conf);
+ }
+
+ @Override
+ public void serviceStart() throws Exception {
+ ShuffleHandler.initializeAndStart(shuffleHandlerConf);
+ containerRunner.setShufflePort(ShuffleHandler.get().getPort());
+ server.start();
+ containerRunner.start();
+ }
+
+ public void serviceStop() throws Exception {
+ containerRunner.stop();
+ server.stop();
+ ShuffleHandler.get().stop();
+ }
+
+ public InetSocketAddress getListenerAddress() {
+ return server.getBindAddress();
+ }
+
+ public int getShufflePort() {
+ return ShuffleHandler.get().getPort();
+ }
+
+
+
+ @Override
+ public void queueContainer(RunContainerRequestProto request) throws IOException {
+ numSubmissions.incrementAndGet();
+ containerRunner.queueContainer(request);
+ }
+
+ @Override
+ public void submitWork(TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+ IOException {
+ numSubmissions.incrementAndGet();
+ containerRunner.submitWork(request);
+ }
+
+ public int getNumSubmissions() {
+ return numSubmissions.get();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
new file mode 100644
index 0000000..10d2952
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
@@ -0,0 +1,82 @@
+/*
+ * 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.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+
+
+public class TezTestServiceProtocolClientImpl implements TezTestServiceProtocolBlockingPB {
+
+ private final Configuration conf;
+ private final InetSocketAddress serverAddr;
+ TezTestServiceProtocolBlockingPB proxy;
+
+
+ public TezTestServiceProtocolClientImpl(Configuration conf, String hostname, int port) {
+ this.conf = conf;
+ this.serverAddr = NetUtils.createSocketAddr(hostname, port);
+ }
+
+ @Override
+ public RunContainerResponseProto runContainer(RpcController controller,
+ RunContainerRequestProto request) throws
+ ServiceException {
+ try {
+ return getProxy().runContainer(null, request);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public TezTestServiceProtocolProtos.SubmitWorkResponseProto submitWork(RpcController controller,
+ TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+ ServiceException {
+ try {
+ return getProxy().submitWork(null, request);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+
+ public TezTestServiceProtocolBlockingPB getProxy() throws IOException {
+ if (proxy == null) {
+ proxy = createProxy();
+ }
+ return proxy;
+ }
+
+ public TezTestServiceProtocolBlockingPB createProxy() throws IOException {
+ TezTestServiceProtocolBlockingPB p;
+ // TODO Fix security
+ RPC.setProtocolEngine(conf, TezTestServiceProtocolBlockingPB.class, ProtobufRpcEngine.class);
+ p = (TezTestServiceProtocolBlockingPB) RPC
+ .getProxy(TezTestServiceProtocolBlockingPB.class, 0, serverAddr, conf);
+ return p;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
new file mode 100644
index 0000000..d7f8444
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
@@ -0,0 +1,133 @@
+/*
+ * 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.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.service.ContainerRunner;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+
+public class TezTestServiceProtocolServerImpl extends AbstractService
+ implements TezTestServiceProtocolBlockingPB {
+
+ private static final Log LOG = LogFactory.getLog(TezTestServiceProtocolServerImpl.class);
+
+ private final ContainerRunner containerRunner;
+ private RPC.Server server;
+ private final AtomicReference<InetSocketAddress> bindAddress;
+
+
+ public TezTestServiceProtocolServerImpl(ContainerRunner containerRunner,
+ AtomicReference<InetSocketAddress> address) {
+ super(TezTestServiceProtocolServerImpl.class.getSimpleName());
+ this.containerRunner = containerRunner;
+ this.bindAddress = address;
+ }
+
+ @Override
+ public RunContainerResponseProto runContainer(RpcController controller,
+ RunContainerRequestProto request) throws
+ ServiceException {
+ LOG.info("Received request: " + request);
+ try {
+ containerRunner.queueContainer(request);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ return RunContainerResponseProto.getDefaultInstance();
+ }
+
+ @Override
+ public SubmitWorkResponseProto submitWork(RpcController controller, TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+ ServiceException {
+ LOG.info("Received submitWork request: " + request);
+ try {
+ containerRunner.submitWork(request);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ return SubmitWorkResponseProto.getDefaultInstance();
+ }
+
+
+ @Override
+ public void serviceStart() {
+ Configuration conf = getConfig();
+
+ int numHandlers = 3;
+ InetSocketAddress addr = new InetSocketAddress(0);
+
+ try {
+ server = createServer(TezTestServiceProtocolBlockingPB.class, addr, conf, numHandlers,
+ TezTestServiceProtocolProtos.TezTestServiceProtocol.newReflectiveBlockingService(this));
+ server.start();
+ } catch (IOException e) {
+ LOG.error("Failed to run RPC Server", e);
+ throw new RuntimeException(e);
+ }
+
+ InetSocketAddress serverBindAddress = NetUtils.getConnectAddress(server);
+ this.bindAddress.set(NetUtils.createSocketAddrForHost(
+ serverBindAddress.getAddress().getCanonicalHostName(),
+ serverBindAddress.getPort()));
+ LOG.info("Instantiated TestTestServiceListener at " + bindAddress);
+ }
+
+ @Override
+ public void serviceStop() {
+ if (server != null) {
+ server.stop();
+ }
+ }
+
+ @InterfaceAudience.Private
+ @VisibleForTesting
+ InetSocketAddress getBindAddress() {
+ return this.bindAddress.get();
+ }
+
+ private RPC.Server createServer(Class<?> pbProtocol, InetSocketAddress addr, Configuration conf,
+ int numHandlers, BlockingService blockingService) throws
+ IOException {
+ RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine.class);
+ RPC.Server server = new RPC.Builder(conf)
+ .setProtocol(pbProtocol)
+ .setInstance(blockingService)
+ .setBindAddress(addr.getHostName())
+ .setPort(0)
+ .setNumHandlers(numHandlers)
+ .build();
+ // TODO Add security.
+ return server;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
new file mode 100644
index 0000000..65588fe
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.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.tez.shufflehandler;
+
+import java.io.FileDescriptor;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.jboss.netty.handler.stream.ChunkedFile;
+
+public class FadvisedChunkedFile extends ChunkedFile {
+
+ private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class);
+
+ private final boolean manageOsCache;
+ private final int readaheadLength;
+ private final ReadaheadPool readaheadPool;
+ private final FileDescriptor fd;
+ private final String identifier;
+
+ private ReadaheadRequest readaheadRequest;
+
+ public FadvisedChunkedFile(RandomAccessFile file, long position, long count,
+ int chunkSize, boolean manageOsCache, int readaheadLength,
+ ReadaheadPool readaheadPool, String identifier) throws IOException {
+ super(file, position, count, chunkSize);
+ this.manageOsCache = manageOsCache;
+ this.readaheadLength = readaheadLength;
+ this.readaheadPool = readaheadPool;
+ this.fd = file.getFD();
+ this.identifier = identifier;
+ }
+
+ @Override
+ public Object nextChunk() throws Exception {
+ if (manageOsCache && readaheadPool != null) {
+ readaheadRequest = readaheadPool
+ .readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength,
+ getEndOffset(), readaheadRequest);
+ }
+ return super.nextChunk();
+ }
+
+ @Override
+ public void close() throws Exception {
+ if (readaheadRequest != null) {
+ readaheadRequest.cancel();
+ }
+ if (manageOsCache && getEndOffset() - getStartOffset() > 0) {
+ try {
+ NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
+ fd,
+ getStartOffset(), getEndOffset() - getStartOffset(),
+ NativeIO.POSIX.POSIX_FADV_DONTNEED);
+ } catch (Throwable t) {
+ LOG.warn("Failed to manage OS cache for " + identifier, t);
+ }
+ }
+ super.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
new file mode 100644
index 0000000..bdffe52
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
@@ -0,0 +1,160 @@
+/*
+ * 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.tez.shufflehandler;
+
+import java.io.FileDescriptor;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.jboss.netty.channel.DefaultFileRegion;
+
+public class FadvisedFileRegion extends DefaultFileRegion {
+
+ private static final Log LOG = LogFactory.getLog(FadvisedFileRegion.class);
+
+ private final boolean manageOsCache;
+ private final int readaheadLength;
+ private final ReadaheadPool readaheadPool;
+ private final FileDescriptor fd;
+ private final String identifier;
+ private final long count;
+ private final long position;
+ private final int shuffleBufferSize;
+ private final boolean shuffleTransferToAllowed;
+ private final FileChannel fileChannel;
+
+ private ReadaheadRequest readaheadRequest;
+
+ public FadvisedFileRegion(RandomAccessFile file, long position, long count,
+ boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool,
+ String identifier, int shuffleBufferSize,
+ boolean shuffleTransferToAllowed) throws IOException {
+ super(file.getChannel(), position, count);
+ this.manageOsCache = manageOsCache;
+ this.readaheadLength = readaheadLength;
+ this.readaheadPool = readaheadPool;
+ this.fd = file.getFD();
+ this.identifier = identifier;
+ this.fileChannel = file.getChannel();
+ this.count = count;
+ this.position = position;
+ this.shuffleBufferSize = shuffleBufferSize;
+ this.shuffleTransferToAllowed = shuffleTransferToAllowed;
+ }
+
+ @Override
+ public long transferTo(WritableByteChannel target, long position)
+ throws IOException {
+ if (manageOsCache && readaheadPool != null) {
+ readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
+ getPosition() + position, readaheadLength,
+ getPosition() + getCount(), readaheadRequest);
+ }
+
+ if(this.shuffleTransferToAllowed) {
+ return super.transferTo(target, position);
+ } else {
+ return customShuffleTransfer(target, position);
+ }
+ }
+
+ /**
+ * This method transfers data using local buffer. It transfers data from
+ * a disk to a local buffer in memory, and then it transfers data from the
+ * buffer to the target. This is used only if transferTo is disallowed in
+ * the configuration file. super.TransferTo does not perform well on Windows
+ * due to a small IO request generated. customShuffleTransfer can control
+ * the size of the IO requests by changing the size of the intermediate
+ * buffer.
+ */
+ @VisibleForTesting
+ long customShuffleTransfer(WritableByteChannel target, long position)
+ throws IOException {
+ long actualCount = this.count - position;
+ if (actualCount < 0 || position < 0) {
+ throw new IllegalArgumentException(
+ "position out of range: " + position +
+ " (expected: 0 - " + (this.count - 1) + ')');
+ }
+ if (actualCount == 0) {
+ return 0L;
+ }
+
+ long trans = actualCount;
+ int readSize;
+ ByteBuffer byteBuffer = ByteBuffer.allocate(this.shuffleBufferSize);
+
+ while(trans > 0L &&
+ (readSize = fileChannel.read(byteBuffer, this.position+position)) > 0) {
+ //adjust counters and buffer limit
+ if(readSize < trans) {
+ trans -= readSize;
+ position += readSize;
+ byteBuffer.flip();
+ } else {
+ //We can read more than we need if the actualCount is not multiple
+ //of the byteBuffer size and file is big enough. In that case we cannot
+ //use flip method but we need to set buffer limit manually to trans.
+ byteBuffer.limit((int)trans);
+ byteBuffer.position(0);
+ position += trans;
+ trans = 0;
+ }
+
+ //write data to the target
+ while(byteBuffer.hasRemaining()) {
+ target.write(byteBuffer);
+ }
+
+ byteBuffer.clear();
+ }
+
+ return actualCount - trans;
+ }
+
+
+ @Override
+ public void releaseExternalResources() {
+ if (readaheadRequest != null) {
+ readaheadRequest.cancel();
+ }
+ super.releaseExternalResources();
+ }
+
+ /**
+ * Call when the transfer completes successfully so we can advise the OS that
+ * we don't need the region to be cached anymore.
+ */
+ public void transferSuccessful() {
+ if (manageOsCache && getCount() > 0) {
+ try {
+ NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
+ fd, getPosition(), getCount(),
+ NativeIO.POSIX.POSIX_FADV_DONTNEED);
+ } catch (Throwable t) {
+ LOG.warn("Failed to manage OS cache for " + identifier, t);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
new file mode 100644
index 0000000..9a51ca0
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
@@ -0,0 +1,199 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.tez.shufflehandler;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+
+class IndexCache {
+
+ private final Configuration conf;
+ private final int totalMemoryAllowed;
+ private AtomicInteger totalMemoryUsed = new AtomicInteger();
+ private static final Log LOG = LogFactory.getLog(IndexCache.class);
+
+ private final ConcurrentHashMap<String,IndexInformation> cache =
+ new ConcurrentHashMap<String,IndexInformation>();
+
+ private final LinkedBlockingQueue<String> queue =
+ new LinkedBlockingQueue<String>();
+
+ public IndexCache(Configuration conf) {
+ this.conf = conf;
+ totalMemoryAllowed = 10 * 1024 * 1024;
+ LOG.info("IndexCache created with max memory = " + totalMemoryAllowed);
+ }
+
+ /**
+ * This method gets the index information for the given mapId and reduce.
+ * It reads the index file into cache if it is not already present.
+ * @param mapId
+ * @param reduce
+ * @param fileName The file to read the index information from if it is not
+ * already present in the cache
+ * @param expectedIndexOwner The expected owner of the index file
+ * @return The Index Information
+ * @throws IOException
+ */
+ public TezIndexRecord getIndexInformation(String mapId, int reduce,
+ Path fileName, String expectedIndexOwner)
+ throws IOException {
+
+ IndexInformation info = cache.get(mapId);
+
+ if (info == null) {
+ info = readIndexFileToCache(fileName, mapId, expectedIndexOwner);
+ } else {
+ synchronized(info) {
+ while (isUnderConstruction(info)) {
+ try {
+ info.wait();
+ } catch (InterruptedException e) {
+ throw new IOException("Interrupted waiting for construction", e);
+ }
+ }
+ }
+ LOG.debug("IndexCache HIT: MapId " + mapId + " found");
+ }
+
+ if (info.mapSpillRecord.size() == 0 ||
+ info.mapSpillRecord.size() <= reduce) {
+ throw new IOException("Invalid request " +
+ " Map Id = " + mapId + " Reducer = " + reduce +
+ " Index Info Length = " + info.mapSpillRecord.size());
+ }
+ return info.mapSpillRecord.getIndex(reduce);
+ }
+
+ private boolean isUnderConstruction(IndexInformation info) {
+ synchronized(info) {
+ return (null == info.mapSpillRecord);
+ }
+ }
+
+ private IndexInformation readIndexFileToCache(Path indexFileName,
+ String mapId,
+ String expectedIndexOwner)
+ throws IOException {
+ IndexInformation info;
+ IndexInformation newInd = new IndexInformation();
+ if ((info = cache.putIfAbsent(mapId, newInd)) != null) {
+ synchronized(info) {
+ while (isUnderConstruction(info)) {
+ try {
+ info.wait();
+ } catch (InterruptedException e) {
+ throw new IOException("Interrupted waiting for construction", e);
+ }
+ }
+ }
+ LOG.debug("IndexCache HIT: MapId " + mapId + " found");
+ return info;
+ }
+ LOG.debug("IndexCache MISS: MapId " + mapId + " not found") ;
+ TezSpillRecord tmp = null;
+ try {
+ tmp = new TezSpillRecord(indexFileName, conf, expectedIndexOwner);
+ } catch (Throwable e) {
+ tmp = new TezSpillRecord(0);
+ cache.remove(mapId);
+ throw new IOException("Error Reading IndexFile", e);
+ } finally {
+ synchronized (newInd) {
+ newInd.mapSpillRecord = tmp;
+ newInd.notifyAll();
+ }
+ }
+ queue.add(mapId);
+
+ if (totalMemoryUsed.addAndGet(newInd.getSize()) > totalMemoryAllowed) {
+ freeIndexInformation();
+ }
+ return newInd;
+ }
+
+ /**
+ * This method removes the map from the cache if index information for this
+ * map is loaded(size>0), index information entry in cache will not be
+ * removed if it is in the loading phrase(size=0), this prevents corruption
+ * of totalMemoryUsed. It should be called when a map output on this tracker
+ * is discarded.
+ * @param mapId The taskID of this map.
+ */
+ public void removeMap(String mapId) {
+ IndexInformation info = cache.get(mapId);
+ if (info == null || ((info != null) && isUnderConstruction(info))) {
+ return;
+ }
+ info = cache.remove(mapId);
+ if (info != null) {
+ totalMemoryUsed.addAndGet(-info.getSize());
+ if (!queue.remove(mapId)) {
+ LOG.warn("Map ID" + mapId + " not found in queue!!");
+ }
+ } else {
+ LOG.info("Map ID " + mapId + " not found in cache");
+ }
+ }
+
+ /**
+ * This method checks if cache and totolMemoryUsed is consistent.
+ * It is only used for unit test.
+ * @return True if cache and totolMemoryUsed is consistent
+ */
+ boolean checkTotalMemoryUsed() {
+ int totalSize = 0;
+ for (IndexInformation info : cache.values()) {
+ totalSize += info.getSize();
+ }
+ return totalSize == totalMemoryUsed.get();
+ }
+
+ /**
+ * Bring memory usage below totalMemoryAllowed.
+ */
+ private synchronized void freeIndexInformation() {
+ while (totalMemoryUsed.get() > totalMemoryAllowed) {
+ String s = queue.remove();
+ IndexInformation info = cache.remove(s);
+ if (info != null) {
+ totalMemoryUsed.addAndGet(-info.getSize());
+ }
+ }
+ }
+
+ private static class IndexInformation {
+ TezSpillRecord mapSpillRecord;
+
+ int getSize() {
+ return mapSpillRecord == null
+ ? 0
+ : mapSpillRecord.size() * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/36e7f854/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
new file mode 100644
index 0000000..cc82d74
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
@@ -0,0 +1,840 @@
+/*
+ * 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.tez.shufflehandler;
+
+import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+import javax.crypto.SecretKey;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.SecureIOUtils;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
+import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+public class ShuffleHandler {
+
+ private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
+
+ public static final String SHUFFLE_HANDLER_LOCAL_DIRS = "tez.shuffle.handler.local-dirs";
+
+ public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache";
+ public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
+
+ public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
+ public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+
+ // pattern to identify errors related to the client closing the socket early
+ // idea borrowed from Netty SslHandler
+ private static final Pattern IGNORABLE_ERROR_MESSAGE = Pattern.compile(
+ "^.*(?:connection.*reset|connection.*closed|broken.*pipe).*$",
+ Pattern.CASE_INSENSITIVE);
+
+ private int port;
+ private final ChannelFactory selector;
+ private final ChannelGroup accepted = new DefaultChannelGroup();
+ protected HttpPipelineFactory pipelineFact;
+ private final int sslFileBufferSize;
+ private final Configuration conf;
+
+ private final ConcurrentMap<String, Boolean> registeredApps = new ConcurrentHashMap<String, Boolean>();
+
+ /**
+ * Should the shuffle use posix_fadvise calls to manage the OS cache during
+ * sendfile
+ */
+ private final boolean manageOsCache;
+ private final int readaheadLength;
+ private final int maxShuffleConnections;
+ private final int shuffleBufferSize;
+ private final boolean shuffleTransferToAllowed;
+ private final ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
+
+ private Map<String,String> userRsrc;
+ private JobTokenSecretManager secretManager;
+
+ // TODO Fix this for tez.
+ public static final String MAPREDUCE_SHUFFLE_SERVICEID =
+ "mapreduce_shuffle";
+
+ public static final String SHUFFLE_PORT_CONFIG_KEY = "tez.shuffle.port";
+ public static final int DEFAULT_SHUFFLE_PORT = 15551;
+
+ // TODO Change configs to remove mapreduce references.
+ public static final String SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED =
+ "mapreduce.shuffle.connection-keep-alive.enable";
+ public static final boolean DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED = false;
+
+ public static final String SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT =
+ "mapreduce.shuffle.connection-keep-alive.timeout";
+ public static final int DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT = 5; //seconds
+
+ public static final String SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE =
+ "mapreduce.shuffle.mapoutput-info.meta.cache.size";
+ public static final int DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE =
+ 1000;
+
+ public static final String CONNECTION_CLOSE = "close";
+
+ public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
+ "mapreduce.shuffle.ssl.file.buffer.size";
+
+ public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
+
+ public static final String MAX_SHUFFLE_CONNECTIONS = "mapreduce.shuffle.max.connections";
+ public static final int DEFAULT_MAX_SHUFFLE_CONNECTIONS = 0; // 0 implies no limit
+
+ public static final String MAX_SHUFFLE_THREADS = "mapreduce.shuffle.max.threads";
+ // 0 implies Netty default of 2 * number of available processors
+ public static final int DEFAULT_MAX_SHUFFLE_THREADS = 0;
+
+ public static final String SHUFFLE_BUFFER_SIZE =
+ "mapreduce.shuffle.transfer.buffer.size";
+ public static final int DEFAULT_SHUFFLE_BUFFER_SIZE = 128 * 1024;
+
+ public static final String SHUFFLE_TRANSFERTO_ALLOWED =
+ "mapreduce.shuffle.transferTo.allowed";
+ public static final boolean DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED = true;
+ public static final boolean WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED =
+ false;
+
+ final boolean connectionKeepAliveEnabled;
+ final int connectionKeepAliveTimeOut;
+ final int mapOutputMetaInfoCacheSize;
+ private static final AtomicBoolean started = new AtomicBoolean(false);
+ private static final AtomicBoolean initing = new AtomicBoolean(false);
+ private static ShuffleHandler INSTANCE;
+
+ @Metrics(about="Shuffle output metrics", context="mapred")
+ static class ShuffleMetrics implements ChannelFutureListener {
+ @Metric("Shuffle output in bytes")
+ MutableCounterLong shuffleOutputBytes;
+ @Metric("# of failed shuffle outputs")
+ MutableCounterInt shuffleOutputsFailed;
+ @Metric("# of succeeeded shuffle outputs")
+ MutableCounterInt shuffleOutputsOK;
+ @Metric("# of current shuffle connections")
+ MutableGaugeInt shuffleConnections;
+
+ @Override
+ public void operationComplete(ChannelFuture future) throws Exception {
+ if (future.isSuccess()) {
+ shuffleOutputsOK.incr();
+ } else {
+ shuffleOutputsFailed.incr();
+ }
+ shuffleConnections.decr();
+ }
+ }
+
+ public ShuffleHandler(Configuration conf) {
+ this.conf = conf;
+ manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
+ DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
+
+ readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
+ DEFAULT_SHUFFLE_READAHEAD_BYTES);
+
+ maxShuffleConnections = conf.getInt(MAX_SHUFFLE_CONNECTIONS,
+ DEFAULT_MAX_SHUFFLE_CONNECTIONS);
+ int maxShuffleThreads = conf.getInt(MAX_SHUFFLE_THREADS,
+ DEFAULT_MAX_SHUFFLE_THREADS);
+ if (maxShuffleThreads == 0) {
+ maxShuffleThreads = 2 * Runtime.getRuntime().availableProcessors();
+ }
+
+ shuffleBufferSize = conf.getInt(SHUFFLE_BUFFER_SIZE,
+ DEFAULT_SHUFFLE_BUFFER_SIZE);
+
+ shuffleTransferToAllowed = conf.getBoolean(SHUFFLE_TRANSFERTO_ALLOWED,
+ (Shell.WINDOWS)?WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED:
+ DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED);
+
+ ThreadFactory bossFactory = new ThreadFactoryBuilder()
+ .setNameFormat("ShuffleHandler Netty Boss #%d")
+ .build();
+ ThreadFactory workerFactory = new ThreadFactoryBuilder()
+ .setNameFormat("ShuffleHandler Netty Worker #%d")
+ .build();
+
+ selector = new NioServerSocketChannelFactory(
+ Executors.newCachedThreadPool(bossFactory),
+ Executors.newCachedThreadPool(workerFactory),
+ maxShuffleThreads);
+
+ sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+ DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+ connectionKeepAliveEnabled =
+ conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED,
+ DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED);
+ connectionKeepAliveTimeOut =
+ Math.max(1, conf.getInt(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+ DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT));
+ mapOutputMetaInfoCacheSize =
+ Math.max(1, conf.getInt(SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE,
+ DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE));
+
+ userRsrc = new ConcurrentHashMap<String,String>();
+ secretManager = new JobTokenSecretManager();
+ }
+
+
+ public void start() throws Exception {
+ ServerBootstrap bootstrap = new ServerBootstrap(selector);
+ try {
+ pipelineFact = new HttpPipelineFactory(conf);
+ } catch (Exception ex) {
+ throw new RuntimeException(ex);
+ }
+ bootstrap.setPipelineFactory(pipelineFact);
+ port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
+ Channel ch = bootstrap.bind(new InetSocketAddress(port));
+ accepted.add(ch);
+ port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+ conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
+ pipelineFact.SHUFFLE.setPort(port);
+ LOG.info("TezShuffleHandler" + " listening on port " + port);
+ }
+
+ public static void initializeAndStart(Configuration conf) throws Exception {
+ if (!initing.getAndSet(true)) {
+ INSTANCE = new ShuffleHandler(conf);
+ INSTANCE.start();
+ started.set(true);
+ }
+ }
+
+ public static ShuffleHandler get() {
+ Preconditions.checkState(started.get(), "ShuffleHandler must be started before invoking started");
+ return INSTANCE;
+ }
+
+ /**
+ * Serialize the shuffle port into a ByteBuffer for use later on.
+ * @param port the port to be sent to the ApplciationMaster
+ * @return the serialized form of the port.
+ */
+ public static ByteBuffer serializeMetaData(int port) throws IOException {
+ //TODO these bytes should be versioned
+ DataOutputBuffer port_dob = new DataOutputBuffer();
+ port_dob.writeInt(port);
+ return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
+ }
+
+ /**
+ * A helper function to deserialize the metadata returned by ShuffleHandler.
+ * @param meta the metadata returned by the ShuffleHandler
+ * @return the port the Shuffle Handler is listening on to serve shuffle data.
+ */
+ public static int deserializeMetaData(ByteBuffer meta) throws IOException {
+ //TODO this should be returning a class not just an int
+ DataInputByteBuffer in = new DataInputByteBuffer();
+ in.reset(meta);
+ int port = in.readInt();
+ return port;
+ }
+
+ /**
+ * A helper function to serialize the JobTokenIdentifier to be sent to the
+ * ShuffleHandler as ServiceData.
+ * @param jobToken the job token to be used for authentication of
+ * shuffle data requests.
+ * @return the serialized version of the jobToken.
+ */
+ public static ByteBuffer serializeServiceData(Token<JobTokenIdentifier> jobToken) throws IOException {
+ //TODO these bytes should be versioned
+ DataOutputBuffer jobToken_dob = new DataOutputBuffer();
+ jobToken.write(jobToken_dob);
+ return ByteBuffer.wrap(jobToken_dob.getData(), 0, jobToken_dob.getLength());
+ }
+
+ static Token<JobTokenIdentifier> deserializeServiceData(ByteBuffer secret) throws IOException {
+ DataInputByteBuffer in = new DataInputByteBuffer();
+ in.reset(secret);
+ Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
+ jt.readFields(in);
+ return jt;
+ }
+
+ public int getPort() {
+ return port;
+ }
+
+ public void registerApplication(String applicationIdString, Token<JobTokenIdentifier> appToken,
+ String user) {
+ Boolean registered = registeredApps.putIfAbsent(applicationIdString, Boolean.valueOf(true));
+ if (registered == null) {
+ recordJobShuffleInfo(applicationIdString, user, appToken);
+ }
+ }
+
+ public void unregisterApplication(String applicationIdString) {
+ removeJobShuffleInfo(applicationIdString);
+ }
+
+
+ public void stop() throws Exception {
+ accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+ if (selector != null) {
+ ServerBootstrap bootstrap = new ServerBootstrap(selector);
+ bootstrap.releaseExternalResources();
+ }
+ if (pipelineFact != null) {
+ pipelineFact.destroy();
+ }
+ }
+
+ protected Shuffle getShuffle(Configuration conf) {
+ return new Shuffle(conf);
+ }
+
+
+ private void addJobToken(String appIdString, String user,
+ Token<JobTokenIdentifier> jobToken) {
+ String jobIdString = appIdString.replace("application", "job");
+ userRsrc.put(jobIdString, user);
+ secretManager.addTokenForJob(jobIdString, jobToken);
+ LOG.info("Added token for " + jobIdString);
+ }
+
+ private void recordJobShuffleInfo(String appIdString, String user,
+ Token<JobTokenIdentifier> jobToken) {
+ addJobToken(appIdString, user, jobToken);
+ }
+
+ private void removeJobShuffleInfo(String appIdString) {
+ secretManager.removeTokenForJob(appIdString);
+ userRsrc.remove(appIdString);
+ }
+
+ class HttpPipelineFactory implements ChannelPipelineFactory {
+
+ final Shuffle SHUFFLE;
+ private SSLFactory sslFactory;
+
+ public HttpPipelineFactory(Configuration conf) throws Exception {
+ SHUFFLE = getShuffle(conf);
+ // TODO Setup SSL Shuffle
+// if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY,
+// MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) {
+// LOG.info("Encrypted shuffle is enabled.");
+// sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+// sslFactory.init();
+// }
+ }
+
+ public void destroy() {
+ if (sslFactory != null) {
+ sslFactory.destroy();
+ }
+ }
+
+ @Override
+ public ChannelPipeline getPipeline() throws Exception {
+ ChannelPipeline pipeline = Channels.pipeline();
+ if (sslFactory != null) {
+ pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
+ }
+ pipeline.addLast("decoder", new HttpRequestDecoder());
+ pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
+ pipeline.addLast("encoder", new HttpResponseEncoder());
+ pipeline.addLast("chunking", new ChunkedWriteHandler());
+ pipeline.addLast("shuffle", SHUFFLE);
+ return pipeline;
+ // TODO factor security manager into pipeline
+ // TODO factor out encode/decode to permit binary shuffle
+ // TODO factor out decode of index to permit alt. models
+ }
+
+ }
+
+ class Shuffle extends SimpleChannelUpstreamHandler {
+
+ private final Configuration conf;
+ private final IndexCache indexCache;
+ private final LocalDirAllocator lDirAlloc =
+ new LocalDirAllocator(SHUFFLE_HANDLER_LOCAL_DIRS);
+ private int port;
+
+ public Shuffle(Configuration conf) {
+ this.conf = conf;
+ indexCache = new IndexCache(conf);
+ this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
+ }
+
+ public void setPort(int port) {
+ this.port = port;
+ }
+
+ private List<String> splitMaps(List<String> mapq) {
+ if (null == mapq) {
+ return null;
+ }
+ final List<String> ret = new ArrayList<String>();
+ for (String s : mapq) {
+ Collections.addAll(ret, s.split(","));
+ }
+ return ret;
+ }
+
+ @Override
+ public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt)
+ throws Exception {
+ if ((maxShuffleConnections > 0) && (accepted.size() >= maxShuffleConnections)) {
+ LOG.info(String.format("Current number of shuffle connections (%d) is " +
+ "greater than or equal to the max allowed shuffle connections (%d)",
+ accepted.size(), maxShuffleConnections));
+ evt.getChannel().close();
+ return;
+ }
+ accepted.add(evt.getChannel());
+ super.channelOpen(ctx, evt);
+
+ }
+
+ @Override
+ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
+ throws Exception {
+ HttpRequest request = (HttpRequest) evt.getMessage();
+ if (request.getMethod() != GET) {
+ sendError(ctx, METHOD_NOT_ALLOWED);
+ return;
+ }
+ // Check whether the shuffle version is compatible
+ if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
+ request.getHeader(ShuffleHeader.HTTP_HEADER_NAME))
+ || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
+ request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) {
+ sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST);
+ }
+ final Map<String,List<String>> q =
+ new QueryStringDecoder(request.getUri()).getParameters();
+ final List<String> keepAliveList = q.get("keepAlive");
+ boolean keepAliveParam = false;
+ if (keepAliveList != null && keepAliveList.size() == 1) {
+ keepAliveParam = Boolean.valueOf(keepAliveList.get(0));
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("KeepAliveParam : " + keepAliveList
+ + " : " + keepAliveParam);
+ }
+ }
+ final List<String> mapIds = splitMaps(q.get("map"));
+ final List<String> reduceQ = q.get("reduce");
+ final List<String> jobQ = q.get("job");
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("RECV: " + request.getUri() +
+ "\n mapId: " + mapIds +
+ "\n reduceId: " + reduceQ +
+ "\n jobId: " + jobQ +
+ "\n keepAlive: " + keepAliveParam);
+ }
+
+ if (mapIds == null || reduceQ == null || jobQ == null) {
+ sendError(ctx, "Required param job, map and reduce", BAD_REQUEST);
+ return;
+ }
+ if (reduceQ.size() != 1 || jobQ.size() != 1) {
+ sendError(ctx, "Too many job/reduce parameters", BAD_REQUEST);
+ return;
+ }
+ int reduceId;
+ String jobId;
+ try {
+ reduceId = Integer.parseInt(reduceQ.get(0));
+ jobId = jobQ.get(0);
+ } catch (NumberFormatException e) {
+ sendError(ctx, "Bad reduce parameter", BAD_REQUEST);
+ return;
+ } catch (IllegalArgumentException e) {
+ sendError(ctx, "Bad job parameter", BAD_REQUEST);
+ return;
+ }
+ final String reqUri = request.getUri();
+ if (null == reqUri) {
+ // TODO? add upstream?
+ sendError(ctx, FORBIDDEN);
+ return;
+ }
+ HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+ try {
+ verifyRequest(jobId, ctx, request, response,
+ new URL("http", "", this.port, reqUri));
+ } catch (IOException e) {
+ LOG.warn("Shuffle failure ", e);
+ sendError(ctx, e.getMessage(), UNAUTHORIZED);
+ return;
+ }
+
+ Map<String, MapOutputInfo> mapOutputInfoMap =
+ new HashMap<String, MapOutputInfo>();
+ Channel ch = evt.getChannel();
+ String user = userRsrc.get(jobId);
+
+ // $x/$user/appcache/$appId/output/$mapId
+ // TODO: Once Shuffle is out of NM, this can use MR APIs to convert
+ // between App and Job
+ String outputBasePathStr = getBaseLocation(jobId, user);
+
+ try {
+ populateHeaders(mapIds, outputBasePathStr, user, reduceId, request,
+ response, keepAliveParam, mapOutputInfoMap);
+ } catch(IOException e) {
+ ch.write(response);
+ LOG.error("Shuffle error in populating headers :", e);
+ String errorMessage = getErrorMessage(e);
+ sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR);
+ return;
+ }
+ ch.write(response);
+ // TODO refactor the following into the pipeline
+ ChannelFuture lastMap = null;
+ for (String mapId : mapIds) {
+ try {
+ MapOutputInfo info = mapOutputInfoMap.get(mapId);
+ if (info == null) {
+ info = getMapOutputInfo(outputBasePathStr, mapId, reduceId, user);
+ }
+ lastMap =
+ sendMapOutput(ctx, ch, user, mapId,
+ reduceId, info);
+ if (null == lastMap) {
+ sendError(ctx, NOT_FOUND);
+ return;
+ }
+ } catch (IOException e) {
+ LOG.error("Shuffle error :", e);
+ String errorMessage = getErrorMessage(e);
+ sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR);
+ return;
+ }
+ }
+ lastMap.addListener(ChannelFutureListener.CLOSE);
+ }
+
+ private String getErrorMessage(Throwable t) {
+ StringBuffer sb = new StringBuffer(t.getMessage());
+ while (t.getCause() != null) {
+ sb.append(t.getCause().getMessage());
+ t = t.getCause();
+ }
+ return sb.toString();
+ }
+
+ private final String USERCACHE_CONSTANT = "usercache";
+ private final String APPCACHE_CONSTANT = "appcache";
+
+ private String getBaseLocation(String jobIdString, String user) {
+ String parts[] = jobIdString.split("_");
+ Preconditions.checkArgument(parts.length == 3, "Invalid jobId. Expecting 3 parts");
+ final ApplicationId appID =
+ ApplicationId.newInstance(Long.parseLong(parts[1]), Integer.parseInt(parts[2]));
+ final String baseStr =
+ USERCACHE_CONSTANT + "/" + user + "/"
+ + APPCACHE_CONSTANT + "/"
+ + ConverterUtils.toString(appID) + "/output" + "/";
+ return baseStr;
+ }
+
+ protected MapOutputInfo getMapOutputInfo(String base, String mapId,
+ int reduce, String user) throws IOException {
+ // Index file
+ Path indexFileName =
+ lDirAlloc.getLocalPathToRead(base + "/file.out.index", conf);
+ TezIndexRecord info =
+ indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+
+ Path mapOutputFileName =
+ lDirAlloc.getLocalPathToRead(base + "/file.out", conf);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(base + " : " + mapOutputFileName + " : " + indexFileName);
+ }
+ MapOutputInfo outputInfo = new MapOutputInfo(mapOutputFileName, info);
+ return outputInfo;
+ }
+
+ protected void populateHeaders(List<String> mapIds, String outputBaseStr,
+ String user, int reduce, HttpRequest request, HttpResponse response,
+ boolean keepAliveParam, Map<String, MapOutputInfo> mapOutputInfoMap)
+ throws IOException {
+
+ long contentLength = 0;
+ for (String mapId : mapIds) {
+ String base = outputBaseStr + mapId;
+ MapOutputInfo outputInfo = getMapOutputInfo(base, mapId, reduce, user);
+ if (mapOutputInfoMap.size() < mapOutputMetaInfoCacheSize) {
+ mapOutputInfoMap.put(mapId, outputInfo);
+ }
+ // Index file
+ Path indexFileName =
+ lDirAlloc.getLocalPathToRead(base + "/file.out.index", conf);
+ TezIndexRecord info =
+ indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+ ShuffleHeader header =
+ new ShuffleHeader(mapId, info.getPartLength(), info.getRawLength(), reduce);
+ DataOutputBuffer dob = new DataOutputBuffer();
+ header.write(dob);
+
+ contentLength += info.getPartLength();
+ contentLength += dob.getLength();
+ }
+
+ // Now set the response headers.
+ setResponseHeaders(response, keepAliveParam, contentLength);
+ }
+
+ protected void setResponseHeaders(HttpResponse response,
+ boolean keepAliveParam, long contentLength) {
+ if (!connectionKeepAliveEnabled && !keepAliveParam) {
+ LOG.info("Setting connection close header...");
+ response.setHeader(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE);
+ } else {
+ response.setHeader(HttpHeaders.Names.CONTENT_LENGTH,
+ String.valueOf(contentLength));
+ response.setHeader(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
+ response.setHeader(HttpHeaders.Values.KEEP_ALIVE, "timeout="
+ + connectionKeepAliveTimeOut);
+ LOG.info("Content Length in shuffle : " + contentLength);
+ }
+ }
+
+ class MapOutputInfo {
+ final Path mapOutputFileName;
+ final TezIndexRecord indexRecord;
+
+ MapOutputInfo(Path mapOutputFileName, TezIndexRecord indexRecord) {
+ this.mapOutputFileName = mapOutputFileName;
+ this.indexRecord = indexRecord;
+ }
+ }
+
+ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
+ HttpRequest request, HttpResponse response, URL requestUri)
+ throws IOException {
+ SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid);
+ if (null == tokenSecret) {
+ LOG.info("Request for unknown token " + appid);
+ throw new IOException("could not find jobid");
+ }
+ // string to encrypt
+ String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri);
+ // hash from the fetcher
+ String urlHashStr =
+ request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH);
+ if (urlHashStr == null) {
+ LOG.info("Missing header hash for " + appid);
+ throw new IOException("fetcher cannot be authenticated");
+ }
+ if (LOG.isDebugEnabled()) {
+ int len = urlHashStr.length();
+ LOG.debug("verifying request. enc_str=" + enc_str + "; hash=..." +
+ urlHashStr.substring(len-len/2, len-1));
+ }
+ // verify - throws exception
+ SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
+ // verification passed - encode the reply
+ String reply =
+ SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8),
+ tokenSecret);
+ response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
+ // Put shuffle version into http header
+ response.setHeader(ShuffleHeader.HTTP_HEADER_NAME,
+ ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+ response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION,
+ ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+ if (LOG.isDebugEnabled()) {
+ int len = reply.length();
+ LOG.debug("Fetcher request verfied. enc_str=" + enc_str + ";reply=" +
+ reply.substring(len-len/2, len-1));
+ }
+ }
+
+ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
+ String user, String mapId, int reduce, MapOutputInfo mapOutputInfo)
+ throws IOException {
+ final TezIndexRecord info = mapOutputInfo.indexRecord;
+ final ShuffleHeader header =
+ new ShuffleHeader(mapId, info.getPartLength(), info.getRawLength(), reduce);
+ final DataOutputBuffer dob = new DataOutputBuffer();
+ header.write(dob);
+ ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ final File spillfile =
+ new File(mapOutputInfo.mapOutputFileName.toString());
+ RandomAccessFile spill;
+ try {
+ spill = SecureIOUtils.openForRandomRead(spillfile, "r", user, null);
+ } catch (FileNotFoundException e) {
+ LOG.info(spillfile + " not found");
+ return null;
+ }
+ ChannelFuture writeFuture;
+ if (ch.getPipeline().get(SslHandler.class) == null) {
+ final FadvisedFileRegion partition = new FadvisedFileRegion(spill,
+ info.getStartOffset(), info.getPartLength(), manageOsCache, readaheadLength,
+ readaheadPool, spillfile.getAbsolutePath(),
+ shuffleBufferSize, shuffleTransferToAllowed);
+ writeFuture = ch.write(partition);
+ writeFuture.addListener(new ChannelFutureListener() {
+ // TODO error handling; distinguish IO/connection failures,
+ // attribute to appropriate spill output
+ @Override
+ public void operationComplete(ChannelFuture future) {
+ if (future.isSuccess()) {
+ partition.transferSuccessful();
+ }
+ partition.releaseExternalResources();
+ }
+ });
+ } else {
+ // HTTPS cannot be done with zero copy.
+ final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
+ info.getStartOffset(), info.getPartLength(), sslFileBufferSize,
+ manageOsCache, readaheadLength, readaheadPool,
+ spillfile.getAbsolutePath());
+ writeFuture = ch.write(chunk);
+ }
+ return writeFuture;
+ }
+
+ protected void sendError(ChannelHandlerContext ctx,
+ HttpResponseStatus status) {
+ sendError(ctx, "", status);
+ }
+
+ protected void sendError(ChannelHandlerContext ctx, String message,
+ HttpResponseStatus status) {
+ HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+ response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+ // Put shuffle version into http header
+ response.setHeader(ShuffleHeader.HTTP_HEADER_NAME,
+ ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+ response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION,
+ ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+ response.setContent(
+ ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+
+ // Close the connection as soon as the error message is sent.
+ ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+ }
+
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+ throws Exception {
+ Channel ch = e.getChannel();
+ Throwable cause = e.getCause();
+ if (cause instanceof TooLongFrameException) {
+ sendError(ctx, BAD_REQUEST);
+ return;
+ } else if (cause instanceof IOException) {
+ if (cause instanceof ClosedChannelException) {
+ LOG.debug("Ignoring closed channel error", cause);
+ return;
+ }
+ String message = String.valueOf(cause.getMessage());
+ if (IGNORABLE_ERROR_MESSAGE.matcher(message).matches()) {
+ LOG.debug("Ignoring client socket close", cause);
+ return;
+ }
+ }
+
+ LOG.error("Shuffle error: ", cause);
+ if (ch.isConnected()) {
+ LOG.error("Shuffle error " + e);
+ sendError(ctx, INTERNAL_SERVER_ERROR);
+ }
+ }
+ }
+}
[09/51] [abbrv] tez git commit: TEZ-2241. Miscellaneous fixes after
last reabse. (sseth)
Posted by ss...@apache.org.
TEZ-2241. Miscellaneous fixes after last reabse. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/b3470a13
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/b3470a13
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/b3470a13
Branch: refs/heads/TEZ-2003
Commit: b3470a131935a44fd8659f9ef700d4ede51d5293
Parents: 9cc2acc
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Mar 26 17:57:55 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java | 2 ++
.../org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java | 6 +++---
.../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java | 6 +++---
.../apache/tez/dag/app/launcher/ContainerLauncherRouter.java | 6 +++---
.../test/java/org/apache/tez/dag/app/MockDAGAppMaster.java | 2 +-
tez-ext-service-tests/pom.xml | 4 ++--
.../tez/dag/app/launcher/TezTestServiceContainerLauncher.java | 6 +++---
.../dag/app/launcher/TezTestServiceNoOpContainerLauncher.java | 6 +++---
.../tez/dag/app/rm/TezTestServiceTaskSchedulerService.java | 7 ++++---
.../dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java | 7 ++++---
.../org/apache/tez/service/MiniTezTestServiceCluster.java | 6 +++---
.../java/org/apache/tez/service/impl/ContainerRunnerImpl.java | 5 +++--
.../test/java/org/apache/tez/service/impl/TezTestService.java | 4 ++--
.../tez/service/impl/TezTestServiceProtocolServerImpl.java | 6 +++---
.../org/apache/tez/shufflehandler/FadvisedChunkedFile.java | 6 +++---
.../org/apache/tez/shufflehandler/FadvisedFileRegion.java | 6 +++---
.../test/java/org/apache/tez/shufflehandler/IndexCache.java | 6 +++---
.../java/org/apache/tez/shufflehandler/ShuffleHandler.java | 6 +++---
.../java/org/apache/tez/tests/TestExternalTezServices.java | 6 +++---
20 files changed, 55 insertions(+), 49 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 774a685..d51686d 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -11,5 +11,6 @@ ALL CHANGES:
TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
+ TEZ-2241. Miscellaneous fixes after last reabse.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index e13cc6b..3798b6f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -66,6 +66,8 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.runtime.api.impl.TezEvent;
import org.apache.tez.common.security.JobTokenSecretManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
@SuppressWarnings("unchecked")
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
index 3704cc4..47688d1 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
@@ -18,14 +18,14 @@ import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.api.TezUncheckedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class TezLocalTaskCommunicatorImpl extends TezTaskCommunicatorImpl {
- private static final Log LOG = LogFactory.getLog(TezLocalTaskCommunicatorImpl.class);
+ private static final Logger LOG = LoggerFactory.getLogger(TezLocalTaskCommunicatorImpl.class);
public TezLocalTaskCommunicatorImpl(
TaskCommunicatorContext taskCommunicatorContext) {
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 92b5b6c..bba06fd 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -23,8 +23,6 @@ import java.util.concurrent.ConcurrentMap;
import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -55,11 +53,13 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
public class TezTaskCommunicatorImpl extends TaskCommunicator {
- private static final Log LOG = LogFactory.getLog(TezTaskCommunicatorImpl.class);
+ private static final Logger LOG = LoggerFactory.getLogger(TezTaskCommunicatorImpl.class);
private static final ContainerTask TASK_FOR_INVALID_JVM = new ContainerTask(
null, true, null, null, false);
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 70b0cbc..dd3571e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -19,8 +19,6 @@ import java.lang.reflect.InvocationTargetException;
import java.net.UnknownHostException;
import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -30,11 +28,13 @@ import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.TaskAttemptListener;
import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class ContainerLauncherRouter extends AbstractService
implements EventHandler<NMCommunicatorEvent> {
- static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
+ static final Logger LOG = LoggerFactory.getLogger(ContainerLauncherImpl.class);
private final ContainerLauncher containerLaunchers[];
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 7f45ee6..b39eee2 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -342,7 +342,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
long startTime = System.nanoTime();
long startCpuTime = threadMxBean.getCurrentThreadCpuTime();
TaskHeartbeatResponse response = taListener.heartbeat(request);
- if (response.shouldDie()) {
+ if (response.isShouldDie()) {
cData.remove();
} else {
cData.nextFromEventId = response.getNextFromEventId();
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index 1113341..907e129 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -28,8 +28,8 @@
<dependencies>
<dependency>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index cb6308c..9d22196 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -18,8 +18,6 @@ import java.net.InetSocketAddress;
import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.ApplicationConstants;
@@ -40,6 +38,8 @@ import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
import org.apache.tez.service.TezTestServiceConfConstants;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class TezTestServiceContainerLauncher extends AbstractService implements ContainerLauncher {
@@ -47,7 +47,7 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
// TODO May need multiple connections per target machine, depending upon how synchronization is handled in the RPC layer
- static final Log LOG = LogFactory.getLog(TezTestServiceContainerLauncher.class);
+ static final Logger LOG = LoggerFactory.getLogger(TezTestServiceContainerLauncher.class);
private final AppContext context;
private final String tokenIdentifier;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
index 8c8e486..977d0d3 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
@@ -14,8 +14,6 @@
package org.apache.tez.dag.app.launcher;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.util.Clock;
@@ -28,10 +26,12 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
import org.apache.tez.dag.app.rm.container.AMContainerEventType;
import org.apache.tez.dag.history.DAGHistoryEvent;
import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class TezTestServiceNoOpContainerLauncher extends AbstractService implements ContainerLauncher {
- static final Log LOG = LogFactory.getLog(TezTestServiceNoOpContainerLauncher.class);
+ static final Logger LOG = LoggerFactory.getLogger(TezTestServiceNoOpContainerLauncher.class);
private final AppContext context;
private final Clock clock;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 872d592..50dfb24 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -28,8 +28,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -41,11 +39,14 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.service.TezTestServiceConfConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
- private static final Log LOG = LogFactory.getLog(TezTestServiceTaskSchedulerService.class);
+ private static final Logger
+ LOG = LoggerFactory.getLogger(TezTestServiceTaskSchedulerService.class);
private final ExecutorService appCallbackExecutor;
private final TaskSchedulerAppCallback appClientDelegate;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index e3385a2..ef983c2 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -23,8 +23,6 @@ import java.util.concurrent.RejectedExecutionException;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.ipc.RemoteException;
@@ -41,11 +39,14 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
import org.apache.tez.util.ProtoConverters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl {
- private static final Log LOG = LogFactory.getLog(TezTestServiceTaskCommunicatorImpl.class);
+ private static final Logger
+ LOG = LoggerFactory.getLogger(TezTestServiceTaskCommunicatorImpl.class);
private final TezTestServiceCommunicator communicator;
private final SubmitWorkRequestProto BASE_SUBMIT_WORK_REQUEST;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
index 0ac0b33..c5ff02d 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
@@ -19,8 +19,6 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
@@ -28,10 +26,12 @@ import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.tez.service.impl.TezTestService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class MiniTezTestServiceCluster extends AbstractService {
- private static final Log LOG = LogFactory.getLog(MiniTezTestServiceCluster.class);
+ private static final Logger LOG = LoggerFactory.getLogger(MiniTezTestServiceCluster.class);
private final File testWorkDir;
private final long availableMemory;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index 379d952..e7c02c8 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
-import org.apache.log4j.Logger;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezTaskUmbilicalProtocol;
import org.apache.tez.common.security.JobTokenIdentifier;
@@ -75,10 +74,12 @@ import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainer
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
import org.apache.tez.util.ProtoConverters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class ContainerRunnerImpl extends AbstractService implements ContainerRunner {
- private static final Logger LOG = Logger.getLogger(ContainerRunnerImpl.class);
+ private static final Logger LOG = LoggerFactory.getLogger(ContainerRunnerImpl.class);
public static final String DAG_NAME_INSTRUMENTED_FAILURES = "InstrumentedFailures";
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
index 855f1b0..322be00 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
@@ -23,16 +23,16 @@ import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.StringUtils;
-import org.apache.log4j.Logger;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.service.ContainerRunner;
import org.apache.tez.shufflehandler.ShuffleHandler;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.slf4j.LoggerFactory;
public class TezTestService extends AbstractService implements ContainerRunner {
- private static final Logger LOG = Logger.getLogger(TezTestService.class);
+ private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(TezTestService.class);
private final Configuration shuffleHandlerConf;
private final int numExecutors;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
index 39d7156..b5d3f83 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
@@ -22,8 +22,6 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingService;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -38,11 +36,13 @@ import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainer
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class TezTestServiceProtocolServerImpl extends AbstractService
implements TezTestServiceProtocolBlockingPB {
- private static final Log LOG = LogFactory.getLog(TezTestServiceProtocolServerImpl.class);
+ private static final Logger LOG = LoggerFactory.getLogger(TezTestServiceProtocolServerImpl.class);
private final ContainerRunner containerRunner;
private RPC.Server server;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
index 65588fe..294add6 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
@@ -18,16 +18,16 @@ import java.io.FileDescriptor;
import java.io.IOException;
import java.io.RandomAccessFile;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.ReadaheadPool;
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.jboss.netty.handler.stream.ChunkedFile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class FadvisedChunkedFile extends ChunkedFile {
- private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class);
+ private static final Logger LOG = LoggerFactory.getLogger(FadvisedChunkedFile.class);
private final boolean manageOsCache;
private final int readaheadLength;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
index bdffe52..e5392d3 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedFileRegion.java
@@ -22,16 +22,16 @@ import java.nio.channels.FileChannel;
import java.nio.channels.WritableByteChannel;
import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.ReadaheadPool;
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.jboss.netty.channel.DefaultFileRegion;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class FadvisedFileRegion extends DefaultFileRegion {
- private static final Log LOG = LogFactory.getLog(FadvisedFileRegion.class);
+ private static final Logger LOG = LoggerFactory.getLogger(FadvisedFileRegion.class);
private final boolean manageOsCache;
private final int readaheadLength;
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
index 9a51ca0..5a45917 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/IndexCache.java
@@ -22,20 +22,20 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.tez.runtime.library.common.Constants;
import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
class IndexCache {
private final Configuration conf;
private final int totalMemoryAllowed;
private AtomicInteger totalMemoryUsed = new AtomicInteger();
- private static final Log LOG = LogFactory.getLog(IndexCache.class);
+ private static final Logger LOG = LoggerFactory.getLogger(IndexCache.class);
private final ConcurrentHashMap<String,IndexInformation> cache =
new ConcurrentHashMap<String,IndexInformation>();
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
index cc82d74..8cbb8c7 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
@@ -51,8 +51,6 @@ import java.util.regex.Pattern;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
@@ -105,10 +103,12 @@ import org.jboss.netty.handler.codec.http.QueryStringDecoder;
import org.jboss.netty.handler.ssl.SslHandler;
import org.jboss.netty.handler.stream.ChunkedWriteHandler;
import org.jboss.netty.util.CharsetUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class ShuffleHandler {
- private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
+ private static final Logger LOG = LoggerFactory.getLogger(ShuffleHandler.class);
public static final String SHUFFLE_HANDLER_LOCAL_DIRS = "tez.shuffle.handler.local-dirs";
http://git-wip-us.apache.org/repos/asf/tez/blob/b3470a13/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index b6a166d..4d0a610 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -20,8 +20,6 @@ import java.io.IOException;
import java.util.Map;
import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -48,10 +46,12 @@ import org.apache.tez.test.MiniTezCluster;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class TestExternalTezServices {
- private static final Log LOG = LogFactory.getLog(TestExternalTezServices.class);
+ private static final Logger LOG = LoggerFactory.getLogger(TestExternalTezServices.class);
private static final String EXT_PUSH_ENTITY_NAME = "ExtServiceTestPush";
[10/51] [abbrv] tez git commit: TEZ-2187. Allow TaskCommunicators to
report failed / killed attempts. (sseth)
Posted by ss...@apache.org.
TEZ-2187. Allow TaskCommunicators to report failed / killed attempts. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/9cc2acc3
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/9cc2acc3
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/9cc2acc3
Branch: refs/heads/TEZ-2003
Commit: 9cc2acc354483d76fab2823b6a537bc6bdb3757e
Parents: 61bcca1
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Mar 10 01:25:39 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/common/TezUtilsInternal.java | 60 +++++++++
.../tez/dag/api/TaskAttemptEndReason.java | 24 ++++
.../records/TaskAttemptTerminationCause.java | 7 +-
.../apache/tez/dag/api/TaskCommunicator.java | 2 +
.../tez/dag/api/TaskCommunicatorContext.java | 13 +-
.../dag/app/TaskAttemptListenerImpTezDag.java | 33 +++++
.../event/TaskAttemptEventAttemptFailed.java | 2 +
.../event/TaskAttemptEventAttemptKilled.java | 47 +++++++
.../dag/app/dag/event/TaskAttemptEventType.java | 3 +-
.../tez/dag/app/dag/impl/TaskAttemptImpl.java | 33 ++++-
.../tez/dag/app/rm/AMSchedulerEventTAEnded.java | 9 +-
.../dag/app/rm/LocalTaskSchedulerService.java | 3 +-
.../dag/app/rm/TaskSchedulerEventHandler.java | 7 +-
.../tez/dag/app/rm/TaskSchedulerService.java | 6 +-
.../dag/app/rm/YarnTaskSchedulerService.java | 8 +-
.../app/TestTaskAttemptListenerImplTezDag.java | 1 +
.../app/TestTaskAttemptListenerImplTezDag2.java | 126 +++++++++++++++++++
.../tez/dag/app/rm/TestContainerReuse.java | 65 +++++-----
.../app/rm/TestLocalTaskSchedulerService.java | 5 +-
.../tez/dag/app/rm/TestTaskScheduler.java | 18 +--
.../rm/TezTestServiceTaskSchedulerService.java | 3 +-
.../TezTestServiceTaskCommunicatorImpl.java | 36 +++++-
.../org/apache/tez/service/ContainerRunner.java | 5 +-
.../tez/service/MiniTezTestServiceCluster.java | 5 +-
.../tez/service/impl/ContainerRunnerImpl.java | 60 +++++++--
.../apache/tez/service/impl/TezTestService.java | 6 +-
.../impl/TezTestServiceProtocolServerImpl.java | 10 +-
.../tez/tests/TestExternalTezServices.java | 29 +++++
29 files changed, 547 insertions(+), 80 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 7726815..774a685 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -10,5 +10,6 @@ ALL CHANGES:
TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
+ TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
index 9c78377..347a4f6 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.log4j.Appender;
import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.records.DAGProtos;
import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
@@ -49,6 +50,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Stopwatch;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
@Private
public class TezUtilsInternal {
@@ -234,4 +236,62 @@ public class TezUtilsInternal {
return sb.toString();
}
+ public static TaskAttemptTerminationCause fromTaskAttemptEndReason(
+ TaskAttemptEndReason taskAttemptEndReason) {
+ if (taskAttemptEndReason == null) {
+ return null;
+ }
+ switch (taskAttemptEndReason) {
+ case COMMUNICATION_ERROR:
+ return TaskAttemptTerminationCause.COMMUNICATION_ERROR;
+ case SERVICE_BUSY:
+ return TaskAttemptTerminationCause.SERVICE_BUSY;
+ case INTERRUPTED_BY_SYSTEM:
+ return TaskAttemptTerminationCause.INTERRUPTED_BY_SYSTEM;
+ case INTERRUPTED_BY_USER:
+ return TaskAttemptTerminationCause.INTERRUPTED_BY_USER;
+ case OTHER:
+ return TaskAttemptTerminationCause.UNKNOWN_ERROR;
+ default:
+ return TaskAttemptTerminationCause.UNKNOWN_ERROR;
+ }
+ }
+
+ public static TaskAttemptEndReason toTaskAttemptEndReason(TaskAttemptTerminationCause cause) {
+ // TODO Post TEZ-2003. Consolidate these states, and mappings.
+ if (cause == null) {
+ return null;
+ }
+ switch (cause) {
+ case COMMUNICATION_ERROR:
+ return TaskAttemptEndReason.COMMUNICATION_ERROR;
+ case SERVICE_BUSY:
+ return TaskAttemptEndReason.SERVICE_BUSY;
+ case INTERRUPTED_BY_SYSTEM:
+ return TaskAttemptEndReason.INTERRUPTED_BY_SYSTEM;
+ case INTERRUPTED_BY_USER:
+ return TaskAttemptEndReason.INTERRUPTED_BY_USER;
+ case UNKNOWN_ERROR:
+ case TERMINATED_BY_CLIENT:
+ case TERMINATED_AT_SHUTDOWN:
+ case INTERNAL_PREEMPTION:
+ case EXTERNAL_PREEMPTION:
+ case TERMINATED_INEFFECTIVE_SPECULATION:
+ case TERMINATED_EFFECTIVE_SPECULATION:
+ case TERMINATED_ORPHANED:
+ case APPLICATION_ERROR:
+ case FRAMEWORK_ERROR:
+ case INPUT_READ_ERROR:
+ case OUTPUT_WRITE_ERROR:
+ case OUTPUT_LOST:
+ case TASK_HEARTBEAT_ERROR:
+ case CONTAINER_LAUNCH_FAILED:
+ case CONTAINER_EXITED:
+ case CONTAINER_STOPPED:
+ case NODE_FAILED:
+ case NODE_DISK_ERROR:
+ default:
+ return TaskAttemptEndReason.OTHER;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
new file mode 100644
index 0000000..96a4768
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
@@ -0,0 +1,24 @@
+/*
+ * 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.tez.dag.api;
+
+// TODO TEZ-2003 Expose as a public API
+public enum TaskAttemptEndReason {
+ COMMUNICATION_ERROR,
+ SERVICE_BUSY,
+ INTERRUPTED_BY_SYSTEM,
+ INTERRUPTED_BY_USER,
+ OTHER
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
index ef0bb33..7112d9e 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
@@ -41,5 +41,10 @@ public enum TaskAttemptTerminationCause {
CONTAINER_STOPPED, // Container stopped or released by Tez
NODE_FAILED, // Node for the container failed
NODE_DISK_ERROR, // Disk failed on the node runnign the task
-
+
+ COMMUNICATION_ERROR, // Equivalent to a launch failure
+ SERVICE_BUSY, // Service rejected the task
+ INTERRUPTED_BY_SYSTEM, // Interrupted by the system. e.g. Pre-emption
+ INTERRUPTED_BY_USER, // Interrupted by the user
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index 82eed20..945091e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -45,6 +45,8 @@ public abstract class TaskCommunicator extends AbstractService {
Credentials credentials,
boolean credentialsChanged, int priority);
+ // TODO TEZ-2003. Are additional APIs required to mark a container as completed ? - for completeness.
+
// TODO TEZ-2003 Remove reference to TaskAttemptID
public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 41675fe..a85fb7f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -14,6 +14,7 @@
package org.apache.tez.dag.api;
+import javax.annotation.Nullable;
import java.io.IOException;
import org.apache.hadoop.security.Credentials;
@@ -37,15 +38,21 @@ public interface TaskCommunicatorContext {
// TODO TEZ-2003 Move to vertex, taskIndex, version
boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
+ // TODO TEZ-2003 Split the heartbeat API to a liveness check and a status update
TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
boolean isKnownContainer(ContainerId containerId);
- // TODO TEZ-2003 Move to vertex, taskIndex, version
+ // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
- // TODO TEZ-2003 Add an API to register task failure - for example, a communication failure.
- // This will have to take into consideration the TA_FAILED event
+ // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
+ void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
+
+ // TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
+ void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
+
+ // TODO TEZ-2003 API. Should a method exist for task succeeded.
// TODO Eventually Add methods to report availability stats to the scheduler.
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 04998c3..e13cc6b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -36,14 +36,17 @@ import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
import org.apache.tez.runtime.api.impl.EventType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
@@ -53,7 +56,10 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.Task;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
import org.apache.tez.dag.app.rm.container.AMContainerTask;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -256,6 +262,33 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
pingContainerHeartbeatHandler(containerId);
}
+ @Override
+ public void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+ String diagnostics) {
+ // Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
+ // and messages from the scheduler will release the container.
+ // TODO TEZ-2003 Maybe consider un-registering here itself, since the task is not active anymore,
+ // instead of waiting for the unregister to flow through the Container.
+ // Fix along the same lines as TEZ-2124 by introducing an explict context.
+ context.getEventHandler().handle(new TaskAttemptEventAttemptKilled(taskAttemptId,
+ diagnostics, TezUtilsInternal.fromTaskAttemptEndReason(
+ taskAttemptEndReason)));
+ }
+
+ @Override
+ public void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+ String diagnostics) {
+ // Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
+ // and messages from the scheduler will release the container.
+ // TODO TEZ-2003 Maybe consider un-registering here itself, since the task is not active anymore,
+ // instead of waiting for the unregister to flow through the Container.
+ // Fix along the same lines as TEZ-2124 by introducing an explict context.
+ context.getEventHandler().handle(new TaskAttemptEventAttemptFailed(taskAttemptId,
+ TaskAttemptEventType.TA_FAILED, diagnostics, TezUtilsInternal.fromTaskAttemptEndReason(
+ taskAttemptEndReason)));
+ }
+
+
/**
* Child checking whether it can commit.
* <p/>
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
index b9c1d09..7ec8921 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
@@ -26,6 +26,8 @@ public class TaskAttemptEventAttemptFailed extends TaskAttemptEvent
private final String diagnostics;
private final TaskAttemptTerminationCause errorCause;
+
+ /* Accepted Types - FAILED, TIMED_OUT */
public TaskAttemptEventAttemptFailed(TezTaskAttemptID id,
TaskAttemptEventType type, String diagnostics, TaskAttemptTerminationCause errorCause) {
super(id, type);
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
new file mode 100644
index 0000000..72e6b07
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.tez.dag.app.dag.event;
+
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+public class TaskAttemptEventAttemptKilled extends TaskAttemptEvent
+ implements DiagnosableEvent, TaskAttemptEventTerminationCauseEvent {
+
+ private final String diagnostics;
+ private final TaskAttemptTerminationCause errorCause;
+ public TaskAttemptEventAttemptKilled(TezTaskAttemptID id,
+ String diagnostics,
+ TaskAttemptTerminationCause errorCause) {
+ super(id, TaskAttemptEventType.TA_KILLED);
+ this.diagnostics = diagnostics;
+ this.errorCause = errorCause;
+ }
+
+ @Override
+ public String getDiagnosticInfo() {
+ return diagnostics;
+ }
+
+ @Override
+ public TaskAttemptTerminationCause getTerminationCause() {
+ return errorCause;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
index 228de71..6ba69e3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
@@ -26,11 +26,12 @@ public enum TaskAttemptEventType {
//Producer:Task, Speculator
TA_SCHEDULE,
- //Producer: TaskAttemptListener
+ //Producer: TaskAttemptListener | Vertex after routing events
TA_STARTED_REMOTELY,
TA_STATUS_UPDATE,
TA_DONE,
TA_FAILED,
+ TA_KILLED, // Generated by TaskCommunicators
TA_TIMED_OUT,
//Producer: Client, Scheduler, On speculation.
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
index 1466817..e70123e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.hadoop.yarn.util.Records;
+import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.common.counters.DAGCounter;
import org.apache.tez.common.counters.TezCounters;
import org.apache.tez.dag.api.TezConfiguration;
@@ -187,6 +188,11 @@ public class TaskAttemptImpl implements TaskAttempt,
private final StateMachine<TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent> stateMachine;
+ // TODO TEZ-2003 We may need some additional state management for STATUS_UPDATES, FAILED, KILLED coming in before
+ // TASK_STARTED_REMOTELY. In case of a PUSH it's more intuitive to send TASK_STARTED_REMOTELY after communicating
+ // with the listening service and getting a response, which in turn can trigger STATUS_UPDATES / FAILED / KILLED
+
+ // TA_KILLED handled the same as TA_KILL_REQUEST. Just a different name indicating a request / already killed.
private static StateMachineFactory
<TaskAttemptImpl, TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent>
stateMachineFactory
@@ -219,6 +225,10 @@ public class TaskAttemptImpl implements TaskAttempt,
new TerminatedBeforeRunningTransition(KILLED_HELPER))
.addTransition(TaskAttemptStateInternal.START_WAIT,
TaskAttemptStateInternal.KILL_IN_PROGRESS,
+ TaskAttemptEventType.TA_KILLED,
+ new TerminatedBeforeRunningTransition(KILLED_HELPER))
+ .addTransition(TaskAttemptStateInternal.START_WAIT,
+ TaskAttemptStateInternal.KILL_IN_PROGRESS,
TaskAttemptEventType.TA_NODE_FAILED,
new NodeFailedBeforeRunningTransition())
.addTransition(TaskAttemptStateInternal.START_WAIT,
@@ -255,6 +265,10 @@ public class TaskAttemptImpl implements TaskAttempt,
new TerminatedWhileRunningTransition(KILLED_HELPER))
.addTransition(TaskAttemptStateInternal.RUNNING,
TaskAttemptStateInternal.KILL_IN_PROGRESS,
+ TaskAttemptEventType.TA_KILLED,
+ new TerminatedWhileRunningTransition(KILLED_HELPER))
+ .addTransition(TaskAttemptStateInternal.RUNNING,
+ TaskAttemptStateInternal.KILL_IN_PROGRESS,
TaskAttemptEventType.TA_NODE_FAILED,
new TerminatedWhileRunningTransition(KILLED_HELPER))
.addTransition(TaskAttemptStateInternal.RUNNING,
@@ -289,6 +303,7 @@ public class TaskAttemptImpl implements TaskAttempt,
TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
TaskAttemptEventType.TA_TIMED_OUT,
TaskAttemptEventType.TA_KILL_REQUEST,
+ TaskAttemptEventType.TA_KILLED,
TaskAttemptEventType.TA_NODE_FAILED,
TaskAttemptEventType.TA_CONTAINER_TERMINATING,
TaskAttemptEventType.TA_OUTPUT_FAILED))
@@ -306,6 +321,7 @@ public class TaskAttemptImpl implements TaskAttempt,
TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
TaskAttemptEventType.TA_TIMED_OUT,
TaskAttemptEventType.TA_KILL_REQUEST,
+ TaskAttemptEventType.TA_KILLED,
TaskAttemptEventType.TA_NODE_FAILED,
TaskAttemptEventType.TA_CONTAINER_TERMINATING,
TaskAttemptEventType.TA_OUTPUT_FAILED))
@@ -320,6 +336,7 @@ public class TaskAttemptImpl implements TaskAttempt,
TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
TaskAttemptEventType.TA_TIMED_OUT,
TaskAttemptEventType.TA_KILL_REQUEST,
+ TaskAttemptEventType.TA_KILLED,
TaskAttemptEventType.TA_NODE_FAILED,
TaskAttemptEventType.TA_CONTAINER_TERMINATING,
TaskAttemptEventType.TA_CONTAINER_TERMINATED,
@@ -335,6 +352,7 @@ public class TaskAttemptImpl implements TaskAttempt,
TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
TaskAttemptEventType.TA_TIMED_OUT,
TaskAttemptEventType.TA_KILL_REQUEST,
+ TaskAttemptEventType.TA_KILLED,
TaskAttemptEventType.TA_NODE_FAILED,
TaskAttemptEventType.TA_CONTAINER_TERMINATING,
TaskAttemptEventType.TA_CONTAINER_TERMINATED,
@@ -353,6 +371,12 @@ public class TaskAttemptImpl implements TaskAttempt,
TaskAttemptStateInternal.SUCCEEDED,
EnumSet.of(TaskAttemptStateInternal.KILLED,
TaskAttemptStateInternal.SUCCEEDED),
+ TaskAttemptEventType.TA_KILLED,
+ new TerminatedAfterSuccessTransition())
+ .addTransition(
+ TaskAttemptStateInternal.SUCCEEDED,
+ EnumSet.of(TaskAttemptStateInternal.KILLED,
+ TaskAttemptStateInternal.SUCCEEDED),
TaskAttemptEventType.TA_NODE_FAILED,
new TerminatedAfterSuccessTransition())
.addTransition(
@@ -424,7 +448,6 @@ public class TaskAttemptImpl implements TaskAttempt,
this.leafVertex = leafVertex;
}
-
@Override
public TezTaskAttemptID getID() {
return attemptId;
@@ -1029,6 +1052,7 @@ public class TaskAttemptImpl implements TaskAttempt,
// Compute node/rack location request even if re-scheduled.
Set<String> racks = new HashSet<String>();
+ // TODO Post TEZ-2003. Allow for a policy in the VMPlugin to define localicty for different attempts.
TaskLocationHint locationHint = ta.getTaskLocationHint();
if (locationHint != null) {
if (locationHint.getRacks() != null) {
@@ -1090,6 +1114,8 @@ public class TaskAttemptImpl implements TaskAttempt,
@Override
public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+ // This transition should not be invoked directly, if a scheduler event has already been sent out.
+ // Sub-classes should be used if a scheduler request has been sent.
ta.setFinishTime();
if (event instanceof DiagnosableEvent) {
@@ -1195,7 +1221,8 @@ public class TaskAttemptImpl implements TaskAttempt,
// Inform the scheduler
if (sendSchedulerEvent()) {
ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
- .getTaskAttemptState(), ta.getVertex().getTaskSchedulerIdentifier()));
+ .getTaskAttemptState(), TezUtilsInternal.toTaskAttemptEndReason(ta.terminationCause),
+ ta.getVertex().getTaskSchedulerIdentifier()));
}
}
}
@@ -1277,7 +1304,7 @@ public class TaskAttemptImpl implements TaskAttempt,
// Inform the Scheduler.
ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
- TaskAttemptState.SUCCEEDED, ta.getVertex().getTaskSchedulerIdentifier()));
+ TaskAttemptState.SUCCEEDED, null, ta.getVertex().getTaskSchedulerIdentifier()));
// Inform the task.
ta.sendEvent(new TaskEventTAUpdate(ta.attemptId,
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
index 2ace642..a775948 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
@@ -18,6 +18,7 @@
package org.apache.tez.dag.app.rm;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
import org.apache.tez.dag.app.dag.TaskAttempt;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -27,14 +28,16 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
private final TaskAttempt attempt;
private final ContainerId containerId;
private final TaskAttemptState state;
+ private final TaskAttemptEndReason taskAttemptEndReason;
private final int schedulerId;
public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId,
- TaskAttemptState state, int schedulerId) {
+ TaskAttemptState state, TaskAttemptEndReason taskAttemptEndReason, int schedulerId) {
super(AMSchedulerEventType.S_TA_ENDED);
this.attempt = attempt;
this.containerId = containerId;
this.state = state;
+ this.taskAttemptEndReason = taskAttemptEndReason;
this.schedulerId = schedulerId;
}
@@ -57,4 +60,8 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
public int getSchedulerId() {
return schedulerId;
}
+
+ public TaskAttemptEndReason getTaskAttemptEndReason() {
+ return taskAttemptEndReason;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
index 72a074f..a234e07 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
@@ -149,7 +150,7 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
}
@Override
- public boolean deallocateTask(Object task, boolean taskSucceeded) {
+ public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason) {
return taskRequestHandler.addDeallocateTaskRequest(task);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 9a8bf16..69763b3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -289,7 +289,9 @@ public class TaskSchedulerEventHandler extends AbstractService
private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) {
TaskAttempt attempt = event.getAttempt();
- boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt, false);
+ // Propagate state and failure cause (if any) when informing the scheduler about the de-allocation.
+ boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()]
+ .deallocateTask(attempt, false, event.getTaskAttemptEndReason());
// use stored value of container id in case the scheduler has removed this
// assignment because the task has been deallocated earlier.
// retroactive case
@@ -311,6 +313,7 @@ public class TaskSchedulerEventHandler extends AbstractService
sendEvent(new AMContainerEventStopRequest(attemptContainerId));
// Inform the Node - the task has asked to be STOPPED / has already
// stopped.
+ // AMNodeImpl blacklisting logic does not account for KILLED attempts.
sendEvent(new AMNodeEventTaskAttemptEnded(appContext.getAllContainers().
get(attemptContainerId).getContainer().getNodeId(), attemptContainerId,
attempt.getID(), event.getState() == TaskAttemptState.FAILED));
@@ -332,7 +335,7 @@ public class TaskSchedulerEventHandler extends AbstractService
}
boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt,
- true);
+ true, null);
if (!wasContainerAllocated) {
LOG.error("De-allocated successful task: " + attempt.getID()
+ ", but TaskScheduler reported no container assigned to task");
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
index 48d5455..07dfcd6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
public abstract class TaskSchedulerService extends AbstractService{
@@ -61,8 +62,9 @@ public abstract class TaskSchedulerService extends AbstractService{
public abstract void allocateTask(Object task, Resource capability,
ContainerId containerId, Priority priority, Object containerSignature,
Object clientCookie);
-
- public abstract boolean deallocateTask(Object task, boolean taskSucceeded);
+
+ /** Plugin writers must ensure to de-allocate a container once it's done, so that it can be collected. */
+ public abstract boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason);
public abstract Object deallocateContainer(ContainerId containerId);
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
index 19902b3..6f897e1 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
@@ -995,10 +996,13 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
* the task to de-allocate.
* @param taskSucceeded
* specify whether the task succeeded or failed.
+ * @param endReason
+ * reason for the task ending
* @return true if a container is assigned to this task.
*/
@Override
- public boolean deallocateTask(Object task, boolean taskSucceeded) {
+ public boolean deallocateTask(Object task, boolean taskSucceeded,
+ TaskAttemptEndReason endReason) {
Map<CookieContainerRequest, Container> assignedContainers = null;
synchronized (this) {
@@ -1188,7 +1192,7 @@ public class YarnTaskSchedulerService extends TaskSchedulerService
CookieContainerRequest request = entry.getValue();
if (request.getPriority().equals(lowestPriNewContainer.getPriority())) {
LOG.info("Resending request for task again: " + task);
- deallocateTask(task, true);
+ deallocateTask(task, true, null);
allocateTask(task, request.getCapability(),
(request.getNodes() == null ? null :
request.getNodes().toArray(new String[request.getNodes().size()])),
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 1572c8b..ab9fafe 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -79,6 +79,7 @@ import org.junit.Test;
import org.mockito.ArgumentCaptor;
@SuppressWarnings("unchecked")
+// TODO TEZ-2003 Rename to TestTezTaskCommunicator
public class TestTaskAttemptListenerImplTezDag {
private ApplicationId appId;
private AppContext appContext;
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
new file mode 100644
index 0000000..934543f
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
@@ -0,0 +1,126 @@
+/*
+ * 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.tez.dag.app;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
+import org.apache.tez.dag.app.rm.container.AMContainer;
+import org.apache.tez.dag.app.rm.container.AMContainerMap;
+import org.apache.tez.dag.app.rm.container.AMContainerTask;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+// TODO TEZ-2003. Rename to TestTaskAttemptListener | whatever TaskAttemptListener is renamed to.
+public class TestTaskAttemptListenerImplTezDag2 {
+
+ @Test(timeout = 5000)
+ public void testTaskAttemptFailedKilled() {
+ ApplicationId appId = ApplicationId.newInstance(1000, 1);
+ ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+ Credentials credentials = new Credentials();
+ AppContext appContext = mock(AppContext.class);
+ EventHandler eventHandler = mock(EventHandler.class);
+ DAG dag = mock(DAG.class);
+ AMContainerMap amContainerMap = mock(AMContainerMap.class);
+ Map<ApplicationAccessType, String> appAcls = new HashMap<ApplicationAccessType, String>();
+ doReturn(eventHandler).when(appContext).getEventHandler();
+ doReturn(dag).when(appContext).getCurrentDAG();
+ doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
+ doReturn(credentials).when(appContext).getAppCredentials();
+ doReturn(appAcls).when(appContext).getApplicationACLs();
+ doReturn(amContainerMap).when(appContext).getAllContainers();
+ NodeId nodeId = NodeId.newInstance("localhost", 0);
+ AMContainer amContainer = mock(AMContainer.class);
+ Container container = mock(Container.class);
+ doReturn(nodeId).when(container).getNodeId();
+ doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
+ doReturn(container).when(amContainer).getContainer();
+
+ TaskAttemptListenerImpTezDag taskAttemptListener =
+ new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
+ mock(ContainerHeartbeatHandler.class), null, null, false);
+
+ TaskSpec taskSpec1 = mock(TaskSpec.class);
+ TezTaskAttemptID taskAttemptId1 = mock(TezTaskAttemptID.class);
+ doReturn(taskAttemptId1).when(taskSpec1).getTaskAttemptID();
+ AMContainerTask amContainerTask1 = new AMContainerTask(taskSpec1, null, null, false, 10);
+
+ TaskSpec taskSpec2 = mock(TaskSpec.class);
+ TezTaskAttemptID taskAttemptId2 = mock(TezTaskAttemptID.class);
+ doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
+ AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec2, null, null, false, 10);
+
+ ContainerId containerId1 = createContainerId(appId, 1);
+ taskAttemptListener.registerRunningContainer(containerId1, 0);
+ taskAttemptListener.registerTaskAttempt(amContainerTask1, containerId1, 0);
+ ContainerId containerId2 = createContainerId(appId, 2);
+ taskAttemptListener.registerRunningContainer(containerId2, 0);
+ taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId2, 0);
+
+
+ taskAttemptListener
+ .taskFailed(taskAttemptId1, TaskAttemptEndReason.COMMUNICATION_ERROR, "Diagnostics1");
+ taskAttemptListener
+ .taskKilled(taskAttemptId2, TaskAttemptEndReason.SERVICE_BUSY, "Diagnostics2");
+
+ ArgumentCaptor<Event> argumentCaptor = ArgumentCaptor.forClass(Event.class);
+ verify(eventHandler, times(2)).handle(argumentCaptor.capture());
+ assertTrue(argumentCaptor.getAllValues().get(0) instanceof TaskAttemptEventAttemptFailed);
+ assertTrue(argumentCaptor.getAllValues().get(1) instanceof TaskAttemptEventAttemptKilled);
+ TaskAttemptEventAttemptFailed failedEvent =
+ (TaskAttemptEventAttemptFailed) argumentCaptor.getAllValues().get(0);
+ TaskAttemptEventAttemptKilled killedEvent =
+ (TaskAttemptEventAttemptKilled) argumentCaptor.getAllValues().get(1);
+
+ assertEquals("Diagnostics1", failedEvent.getDiagnosticInfo());
+ assertEquals(TaskAttemptTerminationCause.COMMUNICATION_ERROR,
+ failedEvent.getTerminationCause());
+
+ assertEquals("Diagnostics2", killedEvent.getDiagnosticInfo());
+ assertEquals(TaskAttemptTerminationCause.SERVICE_BUSY, killedEvent.getTerminationCause());
+ // TODO TEZ-2003. Verify unregistration from the registered list
+ }
+
+ private ContainerId createContainerId(ApplicationId applicationId, int containerIdx) {
+ ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(applicationId, 1);
+ ContainerId containerId = ContainerId.newInstance(appAttemptId, containerIdx);
+ return containerId;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 656bca1..394277a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.isNull;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
@@ -57,6 +58,7 @@ import org.apache.tez.common.MockDNSToSwitchMapping;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.OutputDescriptor;
import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
@@ -92,6 +94,7 @@ import org.apache.tez.runtime.api.impl.OutputSpec;
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.junit.BeforeClass;
import org.junit.Test;
+import org.mockito.internal.matchers.Null;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@@ -223,9 +226,9 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(
- ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED, 0));
+ ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler, times(1)).taskAllocated(
eq(ta31), any(Object.class), eq(containerHost1));
verify(rmClient, times(0)).releaseAssignedContainer(
@@ -235,7 +238,7 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
- TaskAttemptState.SUCCEEDED, 0));
+ TaskAttemptState.SUCCEEDED, null, 0));
long currentTs = System.currentTimeMillis();
Throwable exception = null;
@@ -356,9 +359,9 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
- TaskAttemptState.SUCCEEDED, 0));
+ TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta21), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta21), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(
eq(ta31), any(Object.class), eq(containerHost2));
verify(rmClient, times(1)).releaseAssignedContainer(
@@ -459,9 +462,9 @@ public class TestContainerReuse {
verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(container1));
// Task assigned to container completed successfully. Container should be re-used.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta12), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -469,19 +472,19 @@ public class TestContainerReuse {
// Task assigned to container completed successfully.
// Verify reuse across hosts.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta12), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta12), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta13), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
eventHandler.reset();
// Verify no re-use if a previous task fails.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, null, 0));
drainableAppCallback.drain();
verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class), eq(container1));
- verify(taskScheduler).deallocateTask(eq(ta13), eq(false));
+ verify(taskScheduler).deallocateTask(eq(ta13), eq(false), eq((TaskAttemptEndReason)null));
verify(rmClient).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
eventHandler.reset();
@@ -496,9 +499,9 @@ public class TestContainerReuse {
verify(taskSchedulerEventHandler).taskAllocated(eq(ta14), any(Object.class), eq(container2));
// Task assigned to container completed successfully. No pending requests. Container should be released.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta14), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta14), eq(true), eq((TaskAttemptEndReason)null));
verify(rmClient).releaseAssignedContainer(eq(container2.getId()));
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
eventHandler.reset();
@@ -607,9 +610,9 @@ public class TestContainerReuse {
// First task had profiling on. This container can not be reused further.
taskSchedulerEventHandler.handleEvent(
- new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+ new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta12), any(Object.class),
eq(container1));
verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId()));
@@ -653,9 +656,9 @@ public class TestContainerReuse {
// Verify that the container can not be reused when profiling option is turned on
// Even for 2 tasks having same profiling option can have container reusability.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta13), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta13), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class),
eq(container2));
verify(rmClient, times(1)).releaseAssignedContainer(eq(container2.getId()));
@@ -698,9 +701,9 @@ public class TestContainerReuse {
verify(taskSchedulerEventHandler).taskAllocated(eq(ta15), any(Object.class), eq(container3));
//Ensure task 6 (of vertex 1) is allocated to same container
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta15), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta15), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta16), any(Object.class), eq(container3));
eventHandler.reset();
@@ -811,9 +814,9 @@ public class TestContainerReuse {
// until delay expires.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta11, container1.getId(),
- TaskAttemptState.SUCCEEDED, 0));
+ TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(
eq(ta12), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
@@ -828,7 +831,7 @@ public class TestContainerReuse {
// TA12 completed.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta12, container1.getId(),
- TaskAttemptState.SUCCEEDED, 0));
+ TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
LOG.info("Sleeping to ensure that the scheduling loop runs");
Thread.sleep(3000l);
@@ -946,9 +949,9 @@ public class TestContainerReuse {
// Container should be assigned to task21.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta11, container1.getId(),
- TaskAttemptState.SUCCEEDED, 0));
+ TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler).taskAllocated(
eq(ta21), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
@@ -956,7 +959,7 @@ public class TestContainerReuse {
// Task 2 completes.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta21, container1.getId(),
- TaskAttemptState.SUCCEEDED, 0));
+ TaskAttemptState.SUCCEEDED, null, 0));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
LOG.info("Sleeping to ensure that the scheduling loop runs");
@@ -1065,9 +1068,9 @@ public class TestContainerReuse {
assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
// Task assigned to container completed successfully. Container should be re-used.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta111), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -1077,9 +1080,9 @@ public class TestContainerReuse {
// Task assigned to container completed successfully.
// Verify reuse across hosts.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta112), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta112), eq(true), eq((TaskAttemptEndReason)null));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
eventHandler.reset();
@@ -1118,9 +1121,9 @@ public class TestContainerReuse {
assertEquals(2, assignEvent.getRemoteTaskLocalResources().size());
eventHandler.reset();
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta211), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta211), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta212), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
index 25cf4b5..0a642bb 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
@@ -94,7 +95,7 @@ public class TestLocalTaskSchedulerService {
Task task = mock(Task.class);
taskSchedulerService.allocateTask(task, Resource.newInstance(1024, 1), null, null, Priority.newInstance(1), null, null);
- taskSchedulerService.deallocateTask(task, false);
+ taskSchedulerService.deallocateTask(task, false, null);
// start the RequestHandler, DeallocateTaskRequest has higher priority, so will be processed first
taskSchedulerService.startRequestHandlerThread();
@@ -126,7 +127,7 @@ public class TestLocalTaskSchedulerService {
MockAsyncDelegateRequestHandler requestHandler = taskSchedulerService.getRequestHandler();
requestHandler.drainRequest(1);
- taskSchedulerService.deallocateTask(task, false);
+ taskSchedulerService.deallocateTask(task, false, null);
requestHandler.drainRequest(2);
assertEquals(1, requestHandler.deallocateCount);
assertEquals(1, requestHandler.allocateCount);
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
index dabae67..807e772 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
@@ -172,7 +172,7 @@ public class TestTaskScheduler {
addContainerRequest((CookieContainerRequest) any());
// returned from task requests before allocation happens
- assertFalse(scheduler.deallocateTask(mockTask1, true));
+ assertFalse(scheduler.deallocateTask(mockTask1, true, null));
verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class));
verify(mockRMClient, times(1)).
removeContainerRequest((CookieContainerRequest) any());
@@ -180,7 +180,7 @@ public class TestTaskScheduler {
releaseAssignedContainer((ContainerId) any());
// deallocating unknown task
- assertFalse(scheduler.deallocateTask(mockTask1, true));
+ assertFalse(scheduler.deallocateTask(mockTask1, true, null));
verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class));
verify(mockRMClient, times(1)).
removeContainerRequest((CookieContainerRequest) any());
@@ -325,7 +325,7 @@ public class TestTaskScheduler {
verify(mockRMClient).releaseAssignedContainer(mockCId4);
// deallocate allocated task
- assertTrue(scheduler.deallocateTask(mockTask1, true));
+ assertTrue(scheduler.deallocateTask(mockTask1, true, null));
drainableAppCallback.drain();
verify(mockApp).containerBeingReleased(mockCId1);
verify(mockRMClient).releaseAssignedContainer(mockCId1);
@@ -445,7 +445,7 @@ public class TestTaskScheduler {
verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any());
verify(mockApp).taskAllocated(mockTask4, mockCookie4, mockContainer6);
// deallocate allocated task
- assertTrue(scheduler.deallocateTask(mockTask4, true));
+ assertTrue(scheduler.deallocateTask(mockTask4, true, null));
drainableAppCallback.drain();
verify(mockApp).containerBeingReleased(mockCId6);
verify(mockRMClient).releaseAssignedContainer(mockCId6);
@@ -475,7 +475,7 @@ public class TestTaskScheduler {
removeContainerRequest((CookieContainerRequest) any());
verify(mockRMClient, times(8)).addContainerRequest(
(CookieContainerRequest) any());
- assertFalse(scheduler.deallocateTask(mockTask1, true));
+ assertFalse(scheduler.deallocateTask(mockTask1, true, null));
List<NodeReport> mockUpdatedNodes = mock(List.class);
scheduler.onNodesUpdated(mockUpdatedNodes);
@@ -741,7 +741,7 @@ public class TestTaskScheduler {
verify(mockRMClient).releaseAssignedContainer(mockCId4);
// deallocate allocated task
- assertTrue(scheduler.deallocateTask(mockTask1, true));
+ assertTrue(scheduler.deallocateTask(mockTask1, true, null));
drainableAppCallback.drain();
verify(mockApp).containerBeingReleased(mockCId1);
verify(mockRMClient).releaseAssignedContainer(mockCId1);
@@ -871,7 +871,7 @@ public class TestTaskScheduler {
verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any());
verify(mockApp).taskAllocated(mockTask4, mockCookie4, mockContainer6);
// deallocate allocated task
- assertTrue(scheduler.deallocateTask(mockTask4, true));
+ assertTrue(scheduler.deallocateTask(mockTask4, true, null));
drainableAppCallback.drain();
verify(mockApp).containerBeingReleased(mockCId6);
verify(mockRMClient).releaseAssignedContainer(mockCId6);
@@ -960,8 +960,8 @@ public class TestTaskScheduler {
// container7 allocated to the task with affinity for it
verify(mockApp).taskAllocated(mockTask6, mockCookie6, mockContainer7);
// deallocate allocated task
- assertTrue(scheduler.deallocateTask(mockTask5, true));
- assertTrue(scheduler.deallocateTask(mockTask6, true));
+ assertTrue(scheduler.deallocateTask(mockTask5, true, null));
+ assertTrue(scheduler.deallocateTask(mockTask6, true, null));
drainableAppCallback.drain();
verify(mockApp).containerBeingReleased(mockCId7);
verify(mockApp).containerBeingReleased(mockCId8);
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 5657f86..872d592 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.service.TezTestServiceConfConstants;
@@ -198,7 +199,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
}
@Override
- public boolean deallocateTask(Object task, boolean taskSucceeded) {
+ public boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason) {
ContainerId containerId = runningTasks.remove(task);
if (containerId == null) {
LOG.error("Could not determine ContainerId for task: " + task +
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index a327caf..e3385a2 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -19,16 +19,20 @@ import java.nio.ByteBuffer;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.RejectedExecutionException;
import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
import org.apache.tez.dag.app.TezTestServiceCommunicator;
@@ -83,6 +87,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
@Override
public void serviceStop() {
super.serviceStop();
+ this.communicator.stop();
}
@@ -123,13 +128,15 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
throw new RuntimeException("ContainerInfo not found for container: " + containerId +
", while trying to launch task: " + taskSpec.getTaskAttemptID());
}
+ // Have to register this up front right now. Otherwise, it's possible for the task to start
+ // sending out status/DONE/KILLED/FAILED messages before TAImpl knows how to handle them.
+ getTaskCommunicatorContext()
+ .taskStartedRemotely(taskSpec.getTaskAttemptID(), containerId);
communicator.submitWork(requestProto, host, port,
new TezTestServiceCommunicator.ExecuteRequestCallback<SubmitWorkResponseProto>() {
@Override
public void setResponse(SubmitWorkResponseProto response) {
LOG.info("Successfully launched task: " + taskSpec.getTaskAttemptID());
- getTaskCommunicatorContext()
- .taskStartedRemotely(taskSpec.getTaskAttemptID(), containerId);
}
@Override
@@ -137,6 +144,31 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
// TODO Handle this error. This is where an API on the context to indicate failure / rejection comes in.
LOG.info("Failed to run task: " + taskSpec.getTaskAttemptID() + " on containerId: " +
containerId, t);
+ if (t instanceof ServiceException) {
+ ServiceException se = (ServiceException) t;
+ t = se.getCause();
+ }
+ if (t instanceof RemoteException) {
+ RemoteException re = (RemoteException)t;
+ String message = re.toString();
+ if (message.contains(RejectedExecutionException.class.getName())) {
+ getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
+ TaskAttemptEndReason.SERVICE_BUSY, "Service Busy");
+ } else {
+ getTaskCommunicatorContext()
+ .taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
+ t.toString());
+ }
+ } else {
+ if (t instanceof IOException) {
+ getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
+ TaskAttemptEndReason.COMMUNICATION_ERROR, "Communication Error");
+ } else {
+ getTaskCommunicatorContext()
+ .taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
+ t.getMessage());
+ }
+ }
}
});
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
index 2bca4ed..28c2286 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
@@ -16,12 +16,13 @@ package org.apache.tez.service;
import java.io.IOException;
+import org.apache.tez.dag.api.TezException;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
public interface ContainerRunner {
- void queueContainer(RunContainerRequestProto request) throws IOException;
- void submitWork(SubmitWorkRequestProto request) throws IOException;
+ void queueContainer(RunContainerRequestProto request) throws TezException;
+ void submitWork(SubmitWorkRequestProto request) throws TezException;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
index f47bd67..0ac0b33 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/MiniTezTestServiceCluster.java
@@ -133,7 +133,10 @@ public class MiniTezTestServiceCluster extends AbstractService {
@Override
public void serviceStop() {
- tezTestService.stop();
+ if (tezTestService != null) {
+ tezTestService.stop();
+ tezTestService = null;
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index 25d6030..379d952 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -25,6 +25,7 @@ import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
@@ -58,6 +59,7 @@ import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.TokenCache;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezException;
+import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.task.TaskReporter;
import org.apache.tez.runtime.task.TezTaskRunner;
import org.apache.tez.service.ContainerRunner;
@@ -68,14 +70,18 @@ import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
import org.apache.tez.runtime.task.TezChild;
import org.apache.tez.runtime.task.TezChild.ContainerExecutionResult;
import org.apache.tez.shufflehandler.ShuffleHandler;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
import org.apache.tez.util.ProtoConverters;
public class ContainerRunnerImpl extends AbstractService implements ContainerRunner {
private static final Logger LOG = Logger.getLogger(ContainerRunnerImpl.class);
+ public static final String DAG_NAME_INSTRUMENTED_FAILURES = "InstrumentedFailures";
+
private final ListeningExecutorService executorService;
private final AtomicReference<InetSocketAddress> localAddress;
private final String[] localDirsBase;
@@ -146,10 +152,10 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
* Submit a container which is ready for running.
* The regular pull mechanism will be used to fetch work from the AM
* @param request
- * @throws IOException
+ * @throws TezException
*/
@Override
- public void queueContainer(RunContainerRequestProto request) throws IOException {
+ public void queueContainer(RunContainerRequestProto request) throws TezException {
LOG.info("Queuing container for execution: " + request);
Map<String, String> env = new HashMap<String, String>();
@@ -162,7 +168,11 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
for (int i = 0; i < localDirsBase.length; i++) {
localDirs[i] = createAppSpecificLocalDir(localDirsBase[i], request.getApplicationIdString(),
request.getUser());
- localFs.mkdirs(new Path(localDirs[i]));
+ try {
+ localFs.mkdirs(new Path(localDirs[i]));
+ } catch (IOException e) {
+ throw new TezException(e);
+ }
}
LOG.info("DEBUG: Dirs are: " + Arrays.toString(localDirs));
@@ -175,7 +185,11 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
DataInputBuffer dib = new DataInputBuffer();
byte[] tokenBytes = request.getCredentialsBinary().toByteArray();
dib.reset(tokenBytes, tokenBytes.length);
- credentials.readTokenStorageStream(dib);
+ try {
+ credentials.readTokenStorageStream(dib);
+ } catch (IOException e) {
+ throw new TezException(e);
+ }
Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
@@ -197,13 +211,14 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
* This is intended for a task push from the AM
*
* @param request
- * @throws IOException
+ * @throws org.apache.tez.dag.api.TezException
*/
@Override
- public void submitWork(SubmitWorkRequestProto request) throws
- IOException {
+ public void submitWork(SubmitWorkRequestProto request) throws TezException {
LOG.info("Queuing work for execution: " + request);
+ checkAndThrowExceptionForTests(request);
+
Map<String, String> env = new HashMap<String, String>();
env.putAll(localEnv);
env.put(ApplicationConstants.Environment.USER.name(), request.getUser());
@@ -214,7 +229,11 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
for (int i = 0; i < localDirsBase.length; i++) {
localDirs[i] = createAppSpecificLocalDir(localDirsBase[i], request.getApplicationIdString(),
request.getUser());
- localFs.mkdirs(new Path(localDirs[i]));
+ try {
+ localFs.mkdirs(new Path(localDirs[i]));
+ } catch (IOException e) {
+ throw new TezException(e);
+ }
}
if (LOG.isDebugEnabled()) {
LOG.debug("Dirs are: " + Arrays.toString(localDirs));
@@ -228,7 +247,11 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
DataInputBuffer dib = new DataInputBuffer();
byte[] tokenBytes = request.getCredentialsBinary().toByteArray();
dib.reset(tokenBytes, tokenBytes.length);
- credentials.readTokenStorageStream(dib);
+ try {
+ credentials.readTokenStorageStream(dib);
+ } catch (IOException e) {
+ throw new TezException(e);
+ }
Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
@@ -509,4 +532,23 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
}
}
+
+ private void checkAndThrowExceptionForTests(SubmitWorkRequestProto request) throws TezException {
+ if (!request.getTaskSpec().getDagName().equals(DAG_NAME_INSTRUMENTED_FAILURES)) {
+ return;
+ }
+
+ TaskSpec taskSpec = ProtoConverters.getTaskSpecfromProto(request.getTaskSpec());
+ if (taskSpec.getTaskAttemptID().getTaskID().getId() == 0 &&
+ taskSpec.getTaskAttemptID().getId() == 0) {
+ LOG.info("Simulating Rejected work");
+ throw new RejectedExecutionException(
+ "Simulating Rejected work for taskAttemptId=" + taskSpec.getTaskAttemptID());
+ } else if (taskSpec.getTaskAttemptID().getTaskID().getId() == 1 &&
+ taskSpec.getTaskAttemptID().getId() == 0) {
+ LOG.info("Simulating Task Setup Failure during launch");
+ throw new TezException("Simulating Task Setup Failure during launch for taskAttemptId=" +
+ taskSpec.getTaskAttemptID());
+ }
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
index 012e352..855f1b0 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
@@ -14,7 +14,6 @@
package org.apache.tez.service.impl;
-import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicInteger;
@@ -25,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Logger;
+import org.apache.tez.dag.api.TezException;
import org.apache.tez.service.ContainerRunner;
import org.apache.tez.shufflehandler.ShuffleHandler;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
@@ -108,14 +108,14 @@ public class TezTestService extends AbstractService implements ContainerRunner {
@Override
- public void queueContainer(RunContainerRequestProto request) throws IOException {
+ public void queueContainer(RunContainerRequestProto request) throws TezException {
numSubmissions.incrementAndGet();
containerRunner.queueContainer(request);
}
@Override
public void submitWork(TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
- IOException {
+ TezException {
numSubmissions.incrementAndGet();
containerRunner.submitWork(request);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
index d7f8444..39d7156 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
@@ -30,11 +30,13 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.dag.api.TezException;
import org.apache.tez.service.ContainerRunner;
import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
public class TezTestServiceProtocolServerImpl extends AbstractService
@@ -61,20 +63,20 @@ public class TezTestServiceProtocolServerImpl extends AbstractService
LOG.info("Received request: " + request);
try {
containerRunner.queueContainer(request);
- } catch (IOException e) {
+ } catch (TezException e) {
throw new ServiceException(e);
}
return RunContainerResponseProto.getDefaultInstance();
}
@Override
- public SubmitWorkResponseProto submitWork(RpcController controller, TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+ public SubmitWorkResponseProto submitWork(RpcController controller, SubmitWorkRequestProto request) throws
ServiceException {
LOG.info("Received submitWork request: " + request);
try {
containerRunner.submitWork(request);
- } catch (IOException e) {
- e.printStackTrace();
+ } catch (TezException e) {
+ throw new ServiceException(e);
}
return SubmitWorkResponseProto.getDefaultInstance();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9cc2acc3/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 0ec972b..b6a166d 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -27,16 +27,23 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.ProcessorDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
import org.apache.tez.dag.app.launcher.TezTestServiceNoOpContainerLauncher;
import org.apache.tez.dag.app.rm.TezTestServiceTaskSchedulerService;
import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
import org.apache.tez.examples.HashJoinExample;
import org.apache.tez.examples.JoinDataGen;
import org.apache.tez.examples.JoinValidateConfigured;
+import org.apache.tez.runtime.library.processor.SleepProcessor;
import org.apache.tez.service.MiniTezTestServiceCluster;
+import org.apache.tez.service.impl.ContainerRunnerImpl;
import org.apache.tez.test.MiniTezCluster;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@@ -283,6 +290,28 @@ public class TestExternalTezServices {
PROPS_IN_AM, PROPS_REGULAR_CONTAINERS);
}
+ @Test(timeout = 60000)
+ public void testErrorPropagation() throws TezException, InterruptedException, IOException {
+ runExceptionSimulation();
+ }
+
+
+
+ private void runExceptionSimulation() throws IOException, TezException, InterruptedException {
+ DAG dag = DAG.create(ContainerRunnerImpl.DAG_NAME_INSTRUMENTED_FAILURES);
+ Vertex v =Vertex.create("Vertex1", ProcessorDescriptor.create(SleepProcessor.class.getName()),
+ 3);
+ for (Map.Entry<String, String> prop : PROPS_EXT_SERVICE_PUSH.entrySet()) {
+ v.setConf(prop.getKey(), prop.getValue());
+ }
+ dag.addVertex(v);
+ DAGClient dagClient = sharedTezClient.submitDAG(dag);
+ DAGStatus dagStatus = dagClient.waitForCompletion();
+ assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState());
+ assertEquals(1, dagStatus.getDAGProgress().getFailedTaskAttemptCount());
+ assertEquals(1, dagStatus.getDAGProgress().getKilledTaskAttemptCount());
+
+ }
private void runJoinValidate(String name, int extExpectedCount, Map<String, String> lhsProps,
Map<String, String> rhsProps,
[30/51] [abbrv] tez git commit: TEZ-2443. TaskRunner2 should call
abort, NPEs while cleaning up tasks. (sseth)
Posted by ss...@apache.org.
TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/625220e9
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/625220e9
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/625220e9
Branch: refs/heads/TEZ-2003
Commit: 625220e97a7e912b1976eba95affc07d0d993dee
Parents: b51e271
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue May 12 14:27:42 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../java/org/apache/tez/dag/api/TaskCommunicator.java | 4 ++++
.../tez/runtime/LogicalIOProcessorRuntimeTask.java | 11 ++++++-----
.../main/java/org/apache/tez/runtime/RuntimeTask.java | 2 +-
.../apache/tez/runtime/task/TaskRunner2Callable.java | 13 +++++++------
.../org/apache/tez/runtime/task/TezTaskRunner2.java | 10 ++++++----
6 files changed, 25 insertions(+), 16 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/625220e9/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 5d2e40a..ed72d6b 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -25,5 +25,6 @@ ALL CHANGES:
TEZ-2433. Fixes after rebase 05/08
TEZ-2438. tez-tools version in the branch is incorrect.
TEZ-2434. Allow tasks to be killed in the Runtime.
+ TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/625220e9/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index cadca0c..2651013 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -48,6 +48,10 @@ public abstract class TaskCommunicator extends AbstractService {
// TODO TEZ-2003. Are additional APIs required to mark a container as completed ? - for completeness.
// TODO TEZ-2003 Remove reference to TaskAttemptID
+ // TODO TEZ-2003 This needs some information about why the attempt is being unregistered.
+ // e.g. preempted in which case the task may need to be informed. Alternately as a result of
+ // a failed task.
+ // In case of preemption - a killTask API is likely a better bet than trying to overload this method.
public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
// TODO TEZ-2003 This doesn't necessarily belong here. A server may not start within the AM.
http://git-wip-us.apache.org/repos/asf/tez/blob/625220e9/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
index 8263b3f..de08e56 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
@@ -704,7 +704,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
}
@Override
- public synchronized void abortTask() throws Exception {
+ public synchronized void abortTask() {
if (processor != null) {
processor.abort();
}
@@ -803,6 +803,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
LOG.debug("Num of inputs to be closed={}", initializedInputs.size());
LOG.debug("Num of outputs to be closed={}", initializedOutputs.size());
}
+
// Close processor
if (!processorClosed && processor != null) {
try {
@@ -820,8 +821,8 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
Thread.currentThread().interrupt();
} catch (Throwable e) {
LOG.warn(
- "Ignoring Exception when closing processor(cleanup). Exception class={}, message={}",
- e.getClass().getName(), e.getMessage());
+ "Ignoring Exception when closing processor(cleanup). Exception class={}, message={}" +
+ e.getClass().getName(), e.getMessage(), e);
}
}
@@ -842,7 +843,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
} catch (Throwable e) {
LOG.warn(
"Ignoring exception when closing input {}(cleanup). Exception class={}, message={}",
- srcVertexName, e.getClass().getName(), e.getMessage());
+ srcVertexName, e.getClass().getName(), e.getMessage(), e);
} finally {
LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
.getContext().getTaskVertexName(), srcVertexName, Thread.currentThread().isInterrupted());
@@ -866,7 +867,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
} catch (Throwable e) {
LOG.warn(
"Ignoring exception when closing output {}(cleanup). Exception class={}, message={}",
- destVertexName, e.getClass().getName(), e.getMessage());
+ destVertexName, e.getClass().getName(), e.getMessage(), e);
} finally {
LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
.getContext().getTaskVertexName(), destVertexName, Thread.currentThread().isInterrupted());
http://git-wip-us.apache.org/repos/asf/tez/blob/625220e9/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
index cdfb46a..33c0113 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
@@ -167,5 +167,5 @@ public abstract class RuntimeTask {
taskDone.set(true);
}
- public abstract void abortTask() throws Exception;
+ public abstract void abortTask();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/625220e9/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
index 7315bbd..ab77635 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
@@ -63,26 +63,26 @@ public class TaskRunner2Callable extends CallableWithNdc<TaskRunner2Callable.Tas
if (stopRequested.get() || Thread.currentThread().isInterrupted()) {
return new TaskRunner2CallableResult(null);
}
- LOG.info("Initializing task" + ", taskAttemptId=" + task.getTaskAttemptID());
+ LOG.info("Initializing task" + ", taskAttemptId={}", task.getTaskAttemptID());
task.initialize();
if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) {
- LOG.info("Running task, taskAttemptId=" + task.getTaskAttemptID());
+ LOG.info("Running task, taskAttemptId={}", task.getTaskAttemptID());
task.run();
} else {
- LOG.info("Stopped before running the processor.");
+ LOG.info("Stopped before running the processor taskAttemptId={}", task.getTaskAttemptID());
return new TaskRunner2CallableResult(null);
}
if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) {
- LOG.info("Closing task, taskAttemptId=" + task.getTaskAttemptID());
+ LOG.info("Closing task, taskAttemptId={}", task.getTaskAttemptID());
task.close();
task.setFrameworkCounters();
} else {
- LOG.info("Stopped before closing the processor");
+ LOG.info("Stopped before closing the processor, taskAttemptId={}", task.getTaskAttemptID());
return new TaskRunner2CallableResult(null);
}
- LOG.info("Task completed, taskAttemptId=" + task.getTaskAttemptID() + ", askedToStop=" + stopRequested.get());
+ LOG.info("Task completed, taskAttemptId={}, askedToStop={}", task.getTaskAttemptID(), stopRequested.get());
return new TaskRunner2CallableResult(null);
@@ -115,6 +115,7 @@ public class TaskRunner2Callable extends CallableWithNdc<TaskRunner2Callable.Tas
public void interruptTask() {
// Ensure the task is only interrupted once.
if (!stopRequested.getAndSet(true)) {
+ task.abortTask();
if (ownThread != null) {
ownThread.interrupt();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/625220e9/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index 73e5c76..ffbc6e8 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -250,10 +250,12 @@ public class TezTaskRunner2 {
public void killTask() {
synchronized (this) {
if (isRunningState()) {
- trySettingEndReason(EndReason.KILL_REQUESTED);
- if (taskRunnerCallable != null) {
- taskKillStartTime = System.currentTimeMillis();
- taskRunnerCallable.interruptTask();
+ if (trySettingEndReason(EndReason.KILL_REQUESTED)) {
+ killTaskRequested.set(true);
+ if (taskRunnerCallable != null) {
+ taskKillStartTime = System.currentTimeMillis();
+ taskRunnerCallable.interruptTask();
+ }
}
}
}
[21/51] [abbrv] tez git commit: TEZ-2420. TaskRunner returning before
executing the task. (sseth)
Posted by ss...@apache.org.
TEZ-2420. TaskRunner returning before executing the task. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/7f0b9677
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/7f0b9677
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/7f0b9677
Branch: refs/heads/TEZ-2003
Commit: 7f0b9677e788ff8c608a7d64143f7ce7a61517c1
Parents: e4f7ea0
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed May 6 17:14:56 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:34 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java | 1 +
2 files changed, 2 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/7f0b9677/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index f8a71e8..9b2339f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -21,5 +21,6 @@ ALL CHANGES:
TEZ-2381. Fixes after rebase 04/28.
TEZ-2388. Send dag identifier as part of the fetcher request string.
TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups.
+ TEZ-2420. TaskRunner returning before executing the task.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/7f0b9677/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index 7238d5e..dd4620a 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -105,6 +105,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
Throwable failureCause = null;
if (!Thread.currentThread().isInterrupted()) {
taskFuture = executor.submit(callable);
+ } else {
return isShutdownRequested();
}
try {
[13/51] [abbrv] tez git commit: TEZ-2125. Create a task communicator
for local mode. Allow tasks to run in the AM. (sseth)
Posted by ss...@apache.org.
TEZ-2125. Create a task communicator for local mode. Allow tasks to run
in the AM. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/248a9948
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/248a9948
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/248a9948
Branch: refs/heads/TEZ-2003
Commit: 248a9948aa6287ed2ebf0a2c3894be86067e7e23
Parents: 359eba2
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 16:12:52 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/dag/app/DAGAppMaster.java | 25 ++++---
.../dag/app/TaskAttemptListenerImpTezDag.java | 18 +++--
.../dag/app/TezLocalTaskCommunicatorImpl.java | 46 +++++++++++++
.../tez/dag/app/TezTaskCommunicatorImpl.java | 71 ++++++++------------
.../app/launcher/ContainerLauncherRouter.java | 17 +++--
.../app/launcher/LocalContainerLauncher.java | 31 ++++++---
.../dag/app/rm/TaskSchedulerEventHandler.java | 2 +
.../apache/tez/dag/app/MockDAGAppMaster.java | 3 +-
.../app/TestTaskAttemptListenerImplTezDag.java | 2 +-
.../tez/service/impl/ContainerRunnerImpl.java | 2 +-
.../tez/tests/TestExternalTezServices.java | 57 +++++++++++++---
.../org/apache/tez/runtime/task/TezChild.java | 34 +++++-----
13 files changed, 206 insertions(+), 103 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 1a2264c..76496c9 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -5,5 +5,6 @@ ALL CHANGES:
TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
TEZ-2122. Setup pluggable components at AM/Vertex level.
TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
+ TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index a304b37..43f8794 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -469,7 +469,7 @@ public class DAGAppMaster extends AbstractService {
//service to handle requests to TaskUmbilicalProtocol
taskAttemptListener = createTaskAttemptListener(context,
- taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorClassIdentifiers);
+ taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorClassIdentifiers, isLocal);
addIfService(taskAttemptListener, true);
containerSignatureMatcher = createContainerSignatureMatcher();
@@ -535,7 +535,7 @@ public class DAGAppMaster extends AbstractService {
taskSchedulerEventHandler);
addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
- this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherClassIdentifiers);
+ this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherClassIdentifiers, isLocal);
addIfService(containerLauncherRouter, true);
dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
@@ -1043,9 +1043,13 @@ public class DAGAppMaster extends AbstractService {
}
protected TaskAttemptListener createTaskAttemptListener(AppContext context,
- TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh, String[] taskCommunicatorClasses) {
+ TaskHeartbeatHandler thh,
+ ContainerHeartbeatHandler chh,
+ String[] taskCommunicatorClasses,
+ boolean isLocal) {
TaskAttemptListener lis =
- new TaskAttemptListenerImpTezDag(context, thh, chh,jobTokenSecretManager, taskCommunicatorClasses);
+ new TaskAttemptListenerImpTezDag(context, thh, chh, jobTokenSecretManager,
+ taskCommunicatorClasses, isLocal);
return lis;
}
@@ -1066,10 +1070,12 @@ public class DAGAppMaster extends AbstractService {
return chh;
}
- protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf, String []containerLauncherClasses) throws
+ protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf,
+ String[] containerLauncherClasses,
+ boolean isLocal) throws
UnknownHostException {
- return new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory, containerLauncherClasses);
-
+ return new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory,
+ containerLauncherClasses, isLocal);
}
public ApplicationId getAppID() {
@@ -2389,9 +2395,8 @@ public class DAGAppMaster extends AbstractService {
StringBuilder sb = new StringBuilder();
sb.append("AM Level configured ").append(component).append(": ");
for (int i = 0; i < classIdentifiers.length; i++) {
- sb.append("[").append(i).append(":").append(map.inverse().get(i)).append(":")
- .append(taskSchedulers.inverse().get(i)).append(
- "]");
+ sb.append("[").append(i).append(":").append(map.inverse().get(i))
+ .append(":").append(classIdentifiers[i]).append("]");
if (i != classIdentifiers.length - 1) {
sb.append(",");
}
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 05c4623..c48601c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -98,13 +98,20 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
// TODO TEZ-2003 pre-merge. Remove reference to JobTokenSecretManager.
JobTokenSecretManager jobTokenSecretManager,
- String [] taskCommunicatorClassIdentifiers) {
+ String [] taskCommunicatorClassIdentifiers,
+ boolean isPureLocalMode) {
super(TaskAttemptListenerImpTezDag.class.getName());
this.context = context;
this.taskHeartbeatHandler = thh;
this.containerHeartbeatHandler = chh;
if (taskCommunicatorClassIdentifiers == null || taskCommunicatorClassIdentifiers.length == 0) {
- taskCommunicatorClassIdentifiers = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ if (isPureLocalMode) {
+ taskCommunicatorClassIdentifiers =
+ new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+ } else {
+ taskCommunicatorClassIdentifiers =
+ new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ }
}
this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
@@ -130,11 +137,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier) {
- if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) ||
- taskCommClassIdentifier
- .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+ if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
LOG.info("Using Default Task Communicator");
return new TezTaskCommunicatorImpl(this);
+ } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+ LOG.info("Using Default Local Task Communicator");
+ return new TezLocalTaskCommunicatorImpl(this);
} else {
LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
new file mode 100644
index 0000000..3704cc4
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezLocalTaskCommunicatorImpl.java
@@ -0,0 +1,46 @@
+/*
+ * 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.tez.dag.app;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class TezLocalTaskCommunicatorImpl extends TezTaskCommunicatorImpl {
+
+ private static final Log LOG = LogFactory.getLog(TezLocalTaskCommunicatorImpl.class);
+
+ public TezLocalTaskCommunicatorImpl(
+ TaskCommunicatorContext taskCommunicatorContext) {
+ super(taskCommunicatorContext);
+ }
+
+ @Override
+ protected void startRpcServer() {
+ try {
+ this.address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
+ } catch (UnknownHostException e) {
+ throw new TezUncheckedException(e);
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Not starting TaskAttemptListener RPC in LocalMode");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 77d2e39..ac8da40 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -15,10 +15,8 @@
package org.apache.tez.dag.app;
import java.io.IOException;
-import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.URISyntaxException;
-import java.net.UnknownHostException;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -76,7 +74,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
private final TezTaskUmbilicalProtocol taskUmbilical;
private final String tokenIdentifier;
private final Token<JobTokenIdentifier> sessionToken;
- private InetSocketAddress address;
+ protected InetSocketAddress address;
private Server server;
public static final class ContainerInfo {
@@ -120,10 +118,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
this.sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
}
-
@Override
public void serviceStart() {
-
startRpcServer();
}
@@ -134,45 +130,34 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
protected void startRpcServer() {
Configuration conf = getConfig();
- if (!conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT)) {
- try {
- JobTokenSecretManager jobTokenSecretManager =
- new JobTokenSecretManager();
- jobTokenSecretManager.addTokenForJob(tokenIdentifier, sessionToken);
-
- server = new RPC.Builder(conf)
- .setProtocol(TezTaskUmbilicalProtocol.class)
- .setBindAddress("0.0.0.0")
- .setPort(0)
- .setInstance(taskUmbilical)
- .setNumHandlers(
- conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
- TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
- .setPortRangeConfig(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE)
- .setSecretManager(jobTokenSecretManager).build();
-
- // Enable service authorization?
- if (conf.getBoolean(
- CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
- false)) {
- refreshServiceAcls(conf, new TezAMPolicyProvider());
- }
-
- server.start();
- this.address = NetUtils.getConnectAddress(server);
- LOG.info("Instantiated TezTaskCommunicator RPC at " + this.address);
- } catch (IOException e) {
- throw new TezUncheckedException(e);
- }
- } else {
- try {
- this.address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
- } catch (UnknownHostException e) {
- throw new TezUncheckedException(e);
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("Not starting TaskAttemptListener RPC in LocalMode");
+ try {
+ JobTokenSecretManager jobTokenSecretManager =
+ new JobTokenSecretManager();
+ jobTokenSecretManager.addTokenForJob(tokenIdentifier, sessionToken);
+
+ server = new RPC.Builder(conf)
+ .setProtocol(TezTaskUmbilicalProtocol.class)
+ .setBindAddress("0.0.0.0")
+ .setPort(0)
+ .setInstance(taskUmbilical)
+ .setNumHandlers(
+ conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
+ TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
+ .setPortRangeConfig(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE)
+ .setSecretManager(jobTokenSecretManager).build();
+
+ // Enable service authorization?
+ if (conf.getBoolean(
+ CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
+ false)) {
+ refreshServiceAcls(conf, new TezAMPolicyProvider());
}
+
+ server.start();
+ this.address = NetUtils.getConnectAddress(server);
+ LOG.info("Instantiated TezTaskCommunicator RPC at " + this.address);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 4f9b5bf..70b0cbc 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
-import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
@@ -49,17 +48,24 @@ public class ContainerLauncherRouter extends AbstractService
public ContainerLauncherRouter(Configuration conf, AppContext context,
TaskAttemptListener taskAttemptListener,
String workingDirectory,
- String[] containerLauncherClassIdentifiers) throws UnknownHostException {
+ String[] containerLauncherClassIdentifiers,
+ boolean isPureLocalMode) throws UnknownHostException {
super(ContainerLauncherRouter.class.getName());
if (containerLauncherClassIdentifiers == null || containerLauncherClassIdentifiers.length == 0) {
- containerLauncherClassIdentifiers = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ if (isPureLocalMode) {
+ containerLauncherClassIdentifiers =
+ new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+ } else {
+ containerLauncherClassIdentifiers =
+ new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ }
}
containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
- taskAttemptListener, workingDirectory, conf);
+ taskAttemptListener, workingDirectory, isPureLocalMode, conf);
}
}
@@ -67,6 +73,7 @@ public class ContainerLauncherRouter extends AbstractService
AppContext context,
TaskAttemptListener taskAttemptListener,
String workingDirectory,
+ boolean isPureLocalMode,
Configuration conf) throws
UnknownHostException {
if (containerLauncherClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
@@ -76,7 +83,7 @@ public class ContainerLauncherRouter extends AbstractService
.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Creating LocalContainerLauncher");
return
- new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
+ new LocalContainerLauncher(context, taskAttemptListener, workingDirectory, isPureLocalMode);
} else {
LOG.info("Creating container launcher : " + containerLauncherClassIdentifier);
Class<? extends ContainerLauncher> containerLauncherClazz =
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index 3c27678..7dbf937 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -36,6 +36,7 @@ import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
@@ -90,9 +91,10 @@ public class LocalContainerLauncher extends AbstractService implements
private final AtomicBoolean serviceStopped = new AtomicBoolean(false);
private final String workingDirectory;
private final TaskAttemptListener tal;
- private final Map<String, String> localEnv = new HashMap<String, String>();
+ private final Map<String, String> localEnv;
private final ExecutionContext executionContext;
private int numExecutors;
+ private final boolean isPureLocalMode;
private final ConcurrentHashMap<ContainerId, RunningTaskCallback>
runningContainers =
@@ -112,16 +114,26 @@ public class LocalContainerLauncher extends AbstractService implements
public LocalContainerLauncher(AppContext context,
TaskAttemptListener taskAttemptListener,
- String workingDirectory) throws UnknownHostException {
+ String workingDirectory,
+ boolean isPureLocalMode) throws UnknownHostException {
super(LocalContainerLauncher.class.getName());
this.context = context;
this.tal = taskAttemptListener;
this.workingDirectory = workingDirectory;
- AuxiliaryServiceHelper.setServiceDataIntoEnv(
- ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ByteBuffer.allocate(4).putInt(0), localEnv);
- executionContext = new ExecutionContextImpl(InetAddress.getLocalHost().getHostName());
- // User cannot be set here since it isn't available till a DAG is running.
+ this.isPureLocalMode = isPureLocalMode;
+ if (isPureLocalMode) {
+ localEnv = Maps.newHashMap();
+ AuxiliaryServiceHelper.setServiceDataIntoEnv(
+ ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ByteBuffer.allocate(4).putInt(0), localEnv);
+ } else {
+ localEnv = System.getenv();
+ }
+
+ // Check if the hostname is set in the environment before overriding it.
+ String host = isPureLocalMode ? InetAddress.getLocalHost().getHostName() :
+ System.getenv(Environment.NM_HOST.name());
+ executionContext = new ExecutionContextImpl(host);
}
@Override
@@ -349,7 +361,9 @@ public class LocalContainerLauncher extends AbstractService implements
InterruptedException, TezException, IOException {
Map<String, String> containerEnv = new HashMap<String, String>();
containerEnv.putAll(localEnv);
- containerEnv.put(Environment.USER.name(), context.getUser());
+ // Use the user from env if it's available.
+ String user = isPureLocalMode ? System.getenv(Environment.USER.name()) : context.getUser();
+ containerEnv.put(Environment.USER.name(), user);
long memAvailable;
synchronized (this) { // needed to fix findbugs Inconsistent synchronization warning
@@ -358,8 +372,7 @@ public class LocalContainerLauncher extends AbstractService implements
TezChild tezChild =
TezChild.newTezChild(defaultConf, null, 0, containerId.toString(), tokenIdentifier,
attemptNumber, localDirs, workingDirectory, containerEnv, "", executionContext, credentials,
- memAvailable, context.getUser());
- tezChild.setUmbilical(tezTaskUmbilicalProtocol);
+ memAvailable, context.getUser(), tezTaskUmbilicalProtocol);
return tezChild;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index ae83730..ba46a67 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -434,6 +434,8 @@ public class TaskSchedulerEventHandler extends AbstractService
} else {
customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT);
}
+ LOG.info("ClusterIdentifier for TaskScheduler [" + i + ":" + taskSchedulerClasses[i] + "]=" +
+ customAppIdIdentifier);
taskSchedulers[i] = createTaskScheduler(host, port,
trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index b70b9ea..7f45ee6 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -516,7 +516,8 @@ public class MockDAGAppMaster extends DAGAppMaster {
// use mock container launcher for tests
@Override
protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
- String[] containerLaunchers)
+ String[] containerLaunchers,
+ boolean isLocal)
throws UnknownHostException {
return new ContainerLauncherRouter(containerLauncher);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 200e737..1572c8b 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -125,7 +125,7 @@ public class TestTaskAttemptListenerImplTezDag {
doReturn(container).when(amContainer).getContainer();
taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index 4a6ce33..25d6030 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -282,7 +282,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
request.getContainerIdString(),
request.getTokenIdentifier(), request.getAppAttemptNumber(), workingDir, localDirs,
envMap, objectRegistry, pid,
- executionContext, credentials, memoryAvailable, request.getUser());
+ executionContext, credentials, memoryAvailable, request.getUser(), null);
ContainerExecutionResult result = tezChild.run();
LOG.info("ExecutionTime for Container: " + request.getContainerIdString() + "=" +
sw.stop().elapsedMillis());
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 9c149c6..01c2080 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -40,6 +40,7 @@ import org.apache.tez.service.MiniTezTestServiceCluster;
import org.apache.tez.test.MiniTezCluster;
import org.junit.AfterClass;
import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Test;
public class TestExternalTezServices {
@@ -120,26 +121,23 @@ public class TestExternalTezServices {
confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
-// TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskSchedulerService.class.getName());
confForJobs.setStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
-// TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
-// TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
// Default all jobs to run via the service. Individual tests override this on a per vertex/dag level.
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
// Setup various executor sets
PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
@@ -232,18 +230,55 @@ public class TestExternalTezServices {
@Test(timeout = 60000)
public void testMixed1() throws Exception { // M-ExtService, R-containers
- int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 3 for num reducers.
+ int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 0 for num reducers.
runJoinValidate("Mixed1", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
PROPS_EXT_SERVICE_PUSH, PROPS_REGULAR_CONTAINERS);
}
@Test(timeout = 60000)
public void testMixed2() throws Exception { // M-Containers, R-ExtService
- int expectedExternalSubmissions = 0 + 3; //4 for 4 src files, 3 for num reducers.
+ int expectedExternalSubmissions = 0 + 3; // 3 for num reducers.
runJoinValidate("Mixed2", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
PROPS_REGULAR_CONTAINERS, PROPS_EXT_SERVICE_PUSH);
}
+ @Test(timeout = 60000)
+ public void testMixed3() throws Exception { // M - service, R-AM
+ int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 0 for num reducers (in-AM).
+ runJoinValidate("Mixed3", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
+ PROPS_EXT_SERVICE_PUSH, PROPS_IN_AM);
+ }
+
+ @Test(timeout = 60000)
+ public void testMixed4() throws Exception { // M - containers, R-AM
+ int expectedExternalSubmissions = 0 + 0; // Nothing in external service.
+ runJoinValidate("Mixed4", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
+ PROPS_REGULAR_CONTAINERS, PROPS_IN_AM);
+ }
+
+ @Test(timeout = 60000)
+ public void testMixed5() throws Exception { // M1 - containers, M2-extservice, R-AM
+ int expectedExternalSubmissions = 2 + 0; // 2 for M2
+ runJoinValidate("Mixed5", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
+ PROPS_EXT_SERVICE_PUSH, PROPS_IN_AM);
+ }
+
+
+ @Ignore // Re-activate this after the AM registers the shuffle token with the launcher.
+ @Test(timeout = 60000)
+ public void testMixed6() throws Exception { // M - AM, R - Service
+ int expectedExternalSubmissions = 0 + 3; // 3 for R in service
+ runJoinValidate("Mixed6", expectedExternalSubmissions, PROPS_IN_AM,
+ PROPS_IN_AM, PROPS_EXT_SERVICE_PUSH);
+ }
+
+ @Test(timeout = 60000)
+ public void testMixed7() throws Exception { // M - AM, R - Containers
+ int expectedExternalSubmissions = 0; // Nothing in ext service
+ runJoinValidate("Mixed7", expectedExternalSubmissions, PROPS_IN_AM,
+ PROPS_IN_AM, PROPS_REGULAR_CONTAINERS);
+ }
+
private void runJoinValidate(String name, int extExpectedCount, Map<String, String> lhsProps,
Map<String, String> rhsProps,
http://git-wip-us.apache.org/repos/asf/tez/blob/248a9948/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index e6ef5e2..32da8fb 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -67,6 +67,7 @@ import org.apache.tez.dag.utils.RelocalizationUtils;
import org.apache.tez.runtime.api.ExecutionContext;
import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -96,7 +97,6 @@ public class TezChild {
private final int amHeartbeatInterval;
private final long sendCounterInterval;
private final int maxEventsToGet;
- private final boolean isLocal;
private final String workingDir;
private final ListeningExecutorService executor;
@@ -111,9 +111,10 @@ public class TezChild {
private final String user;
private Multimap<String, String> startedInputsMap = HashMultimap.create();
+ private final boolean ownUmbilical;
+ private final TezTaskUmbilicalProtocol umbilical;
private TaskReporter taskReporter;
- private TezTaskUmbilicalProtocol umbilical;
private int taskCount = 0;
private TezVertexID lastVertexID;
@@ -122,7 +123,7 @@ public class TezChild {
Map<String, String> serviceProviderEnvMap,
ObjectRegistryImpl objectRegistry, String pid,
ExecutionContext executionContext,
- Credentials credentials, long memAvailable, String user)
+ Credentials credentials, long memAvailable, String user, TezTaskUmbilicalProtocol umbilical)
throws IOException, InterruptedException {
this.defaultConf = conf;
this.containerIdString = containerIdentifier;
@@ -136,6 +137,8 @@ public class TezChild {
this.memAvailable = memAvailable;
this.user = user;
+ LOG.info("TezChild created with umbilical: " + umbilical);
+
getTaskMaxSleepTime = defaultConf.getInt(
TezConfiguration.TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX,
TezConfiguration.TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX_DEFAULT);
@@ -164,25 +167,27 @@ public class TezChild {
}
}
- this.isLocal = defaultConf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
- TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
UserGroupInformation taskOwner = UserGroupInformation.createRemoteUser(tokenIdentifier);
Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
serviceConsumerMetadata.put(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
TezCommonUtils.convertJobTokenToBytes(jobToken));
- if (!isLocal) {
+ if (umbilical == null) {
final InetSocketAddress address = NetUtils.createSocketAddrForHost(host, port);
SecurityUtil.setTokenService(jobToken, address);
taskOwner.addToken(jobToken);
- umbilical = taskOwner.doAs(new PrivilegedExceptionAction<TezTaskUmbilicalProtocol>() {
+ this.umbilical = taskOwner.doAs(new PrivilegedExceptionAction<TezTaskUmbilicalProtocol>() {
@Override
public TezTaskUmbilicalProtocol run() throws Exception {
return RPC.getProxy(TezTaskUmbilicalProtocol.class,
TezTaskUmbilicalProtocol.versionID, address, defaultConf);
}
});
+ ownUmbilical = true;
+ } else {
+ this.umbilical = umbilical;
+ ownUmbilical = false;
}
}
@@ -368,7 +373,7 @@ public class TezChild {
if (taskReporter != null) {
taskReporter.shutdown();
}
- if (!isLocal) {
+ if (ownUmbilical) {
RPC.stopProxy(umbilical);
// TODO Temporary change. Revert. Ideally, move this over to the main method in TezChild if possible.
// LogManager.shutdown();
@@ -376,12 +381,6 @@ public class TezChild {
}
}
- public void setUmbilical(TezTaskUmbilicalProtocol tezTaskUmbilicalProtocol){
- if(tezTaskUmbilicalProtocol != null){
- this.umbilical = tezTaskUmbilicalProtocol;
- }
- }
-
public static class ContainerExecutionResult {
public static enum ExitStatus {
SUCCESS(0),
@@ -436,7 +435,8 @@ public class TezChild {
public static TezChild newTezChild(Configuration conf, String host, int port, String containerIdentifier,
String tokenIdentifier, int attemptNumber, String[] localDirs, String workingDirectory,
Map<String, String> serviceProviderEnvMap, @Nullable String pid,
- ExecutionContext executionContext, Credentials credentials, long memAvailable, String user)
+ ExecutionContext executionContext, Credentials credentials, long memAvailable, String user,
+ TezTaskUmbilicalProtocol tezUmbilical)
throws IOException, InterruptedException, TezException {
// Pull in configuration specified for the session.
@@ -449,7 +449,7 @@ public class TezChild {
return new TezChild(conf, host, port, containerIdentifier, tokenIdentifier,
attemptNumber, workingDirectory, localDirs, serviceProviderEnvMap, objectRegistry, pid,
- executionContext, credentials, memAvailable, user);
+ executionContext, credentials, memAvailable, user, tezUmbilical);
}
public static void main(String[] args) throws IOException, InterruptedException, TezException {
@@ -483,7 +483,7 @@ public class TezChild {
tokenIdentifier, attemptNumber, localDirs, System.getenv(Environment.PWD.name()),
System.getenv(), pid, new ExecutionContextImpl(System.getenv(Environment.NM_HOST.name())),
credentials, Runtime.getRuntime().maxMemory(), System
- .getenv(ApplicationConstants.Environment.USER.toString()));
+ .getenv(ApplicationConstants.Environment.USER.toString()), null);
tezChild.run();
}
[15/51] [abbrv] tez git commit: TEZ-2175. Task priority should be
available to the TaskCommunicator plugin. (sseth)
Posted by ss...@apache.org.
TEZ-2175. Task priority should be available to the TaskCommunicator plugin. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/61bcca16
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/61bcca16
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/61bcca16
Branch: refs/heads/TEZ-2003
Commit: 61bcca168c5aafa96de0c9df609aec3ebdfbe80b
Parents: 6db0fde
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Mar 10 00:47:07 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../src/main/java/org/apache/tez/dag/api/TaskCommunicator.java | 2 +-
.../org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java | 2 +-
.../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java | 3 ++-
.../dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java | 5 +++--
5 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/61bcca16/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 5c5fd8e..7726815 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -9,5 +9,6 @@ ALL CHANGES:
TEZ-2131. Add additional tests for tasks running in the AM.
TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
+ TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/61bcca16/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index c9f85e0..82eed20 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -43,7 +43,7 @@ public abstract class TaskCommunicator extends AbstractService {
public abstract void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
Map<String, LocalResource> additionalResources,
Credentials credentials,
- boolean credentialsChanged);
+ boolean credentialsChanged, int priority);
// TODO TEZ-2003 Remove reference to TaskAttemptID
public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
http://git-wip-us.apache.org/repos/asf/tez/blob/61bcca16/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index c48601c..04998c3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -359,7 +359,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
taskCommunicators[taskCommId].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
- amContainerTask.haveCredentialsChanged());
+ amContainerTask.haveCredentialsChanged(), amContainerTask.getPriority());
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/61bcca16/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index ac8da40..92b5b6c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -196,7 +196,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
@Override
public void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
Map<String, LocalResource> additionalResources,
- Credentials credentials, boolean credentialsChanged) {
+ Credentials credentials, boolean credentialsChanged,
+ int priority) {
ContainerInfo containerInfo = registeredContainers.get(containerId);
Preconditions.checkNotNull(containerInfo,
http://git-wip-us.apache.org/repos/asf/tez/blob/61bcca16/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index 78cdcde..a327caf 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -100,9 +100,10 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
public void registerRunningTaskAttempt(final ContainerId containerId, final TaskSpec taskSpec,
Map<String, LocalResource> additionalResources,
Credentials credentials,
- boolean credentialsChanged) {
+ boolean credentialsChanged,
+ int priority) {
super.registerRunningTaskAttempt(containerId, taskSpec, additionalResources, credentials,
- credentialsChanged);
+ credentialsChanged, priority);
SubmitWorkRequestProto requestProto = null;
try {
requestProto = constructSubmitWorkRequest(containerId, taskSpec);
[38/51] [abbrv] tez git commit: TEZ-2005. Define basic interface for
pluggable TaskScheduler. (sseth)
Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 506e991..7d209bc 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -20,14 +20,10 @@ import java.util.List;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Ints;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -37,30 +33,24 @@ import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
-import org.apache.tez.dag.app.AppContext;
import org.apache.tez.service.TezTestServiceConfConstants;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
+public class TezTestServiceTaskSchedulerService extends TaskScheduler {
private static final Logger
LOG = LoggerFactory.getLogger(TezTestServiceTaskSchedulerService.class);
- private final ExecutorService appCallbackExecutor;
- private final TaskSchedulerAppCallback appClientDelegate;
- private final AppContext appContext;
private final List<String> serviceHosts;
private final ContainerFactory containerFactory;
private final Random random = new Random();
// Currently all services must be running on the same port.
private final int containerPort;
- private final String clientHostname;
- private final int clientPort;
- private final String trackingUrl;
- private final AtomicBoolean isStopped = new AtomicBoolean(false);
private final ConcurrentMap<Object, ContainerId> runningTasks =
new ConcurrentHashMap<Object, ContainerId>();
@@ -77,20 +67,14 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
// Not registering with the RM. Assuming the main TezScheduler will always run (except local mode),
// and take care of YARN registration.
- public TezTestServiceTaskSchedulerService(TaskSchedulerAppCallback appClient,
- AppContext appContext,
- String clientHostname, int clientPort,
- String trackingUrl,
- long customAppIdIdentifier,
- Configuration conf) {
+ public TezTestServiceTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) {
// Accepting configuration here to allow setting up fields as final
- super(TezTestServiceTaskSchedulerService.class.getName());
- this.appCallbackExecutor = createAppCallbackExecutorService();
- this.appClientDelegate = createAppCallbackDelegate(appClient);
- this.appContext = appContext;
+ super(taskSchedulerContext);
this.serviceHosts = new LinkedList<String>();
- this.containerFactory = new ContainerFactory(appContext, customAppIdIdentifier);
+ this.containerFactory = new ContainerFactory(taskSchedulerContext.getApplicationAttemptId(),
+ taskSchedulerContext.getCustomClusterIdentifier());
+ Configuration conf = taskSchedulerContext.getInitialConfiguration();
this.memoryPerInstance = conf
.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, -1);
Preconditions.checkArgument(memoryPerInstance > 0,
@@ -112,10 +96,6 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
Preconditions.checkArgument(executorsPerInstance > 0,
TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT + " must be configured");
- this.clientHostname = clientHostname;
- this.clientPort = clientPort;
- this.trackingUrl = trackingUrl;
-
int memoryPerContainer = (int) (memoryPerInstance / (float) executorsPerInstance);
int coresPerContainer = (int) (coresPerInstance / (float) executorsPerInstance);
this.resourcePerContainer = Resource.newInstance(memoryPerContainer, coresPerContainer);
@@ -138,13 +118,6 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
}
@Override
- public void serviceStop() {
- if (!this.isStopped.getAndSet(true)) {
- appCallbackExecutor.shutdownNow();
- }
- }
-
- @Override
public Resource getAvailableResources() {
// TODO This needs information about all running executors, and the amount of memory etc available across the cluster.
return Resource
@@ -185,7 +158,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
Container container =
containerFactory.createContainer(resourcePerContainer, priority, host, containerPort);
runningTasks.put(task, container.getId());
- appClientDelegate.taskAllocated(task, clientCookie, container);
+ getContext().taskAllocated(task, clientCookie, container);
}
@@ -196,7 +169,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
Container container =
containerFactory.createContainer(resourcePerContainer, priority, host, containerPort);
runningTasks.put(task, container.getId());
- appClientDelegate.taskAllocated(task, clientCookie, container);
+ getContext().taskAllocated(task, clientCookie, container);
}
@Override
@@ -208,7 +181,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
" The query may hang since this \"unknown\" container is now taking up a slot permanently");
return false;
}
- appClientDelegate.containerBeingReleased(containerId);
+ getContext().containerBeingReleased(containerId);
return true;
}
@@ -229,17 +202,6 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
return true;
}
- private ExecutorService createAppCallbackExecutorService() {
- return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
- .setNameFormat("TezTestTaskSchedulerAppCaller").setDaemon(true).build());
- }
-
- private TaskSchedulerAppCallback createAppCallbackDelegate(
- TaskSchedulerAppCallback realAppClient) {
- return new TaskSchedulerAppCallbackWrapper(realAppClient,
- appCallbackExecutor);
- }
-
private String selectHost(String[] requestedHosts) {
String host;
if (requestedHosts != null && requestedHosts.length > 0) {
@@ -257,12 +219,12 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
AtomicInteger nextId;
final ApplicationAttemptId customAppAttemptId;
- public ContainerFactory(AppContext appContext, long appIdLong) {
+ public ContainerFactory(ApplicationAttemptId appAttemptId, long appIdLong) {
this.nextId = new AtomicInteger(1);
ApplicationId appId = ApplicationId
- .newInstance(appIdLong, appContext.getApplicationAttemptId().getApplicationId().getId());
+ .newInstance(appIdLong, appAttemptId.getApplicationId().getId());
this.customAppAttemptId = ApplicationAttemptId
- .newInstance(appId, appContext.getApplicationAttemptId().getAttemptId());
+ .newInstance(appId, appAttemptId.getAttemptId());
}
public Container createContainer(Resource capability, Priority priority, String hostname, int port) {
[44/51] [abbrv] tez git commit: TEZ-2652. Cleanup the way services
are specified for an AM and vertices. (sseth)
Posted by ss...@apache.org.
TEZ-2652. Cleanup the way services are specified for an AM and vertices. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/5afbef1b
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/5afbef1b
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/5afbef1b
Branch: refs/heads/TEZ-2003
Commit: 5afbef1b3ce35f8eb73d36fad5e150813defb8a1
Parents: f5da29d
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jul 28 14:56:20 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:11 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../java/org/apache/tez/client/TezClient.java | 81 ++++++++-
.../org/apache/tez/client/TezClientUtils.java | 56 +++++-
.../main/java/org/apache/tez/dag/api/DAG.java | 48 +++++-
.../apache/tez/dag/api/DagTypeConverters.java | 93 +++++++++-
.../tez/dag/api/NamedEntityDescriptor.java | 33 ++++
.../apache/tez/dag/api/TezConfiguration.java | 31 ----
.../org/apache/tez/dag/api/TezConstants.java | 11 +-
.../java/org/apache/tez/dag/api/Vertex.java | 110 +++++++++++-
.../api/ContainerLauncherDescriptor.java | 32 ++++
.../api/ServicePluginsDescriptor.java | 96 +++++++++++
.../api/TaskCommunicatorDescriptor.java | 33 ++++
.../api/TaskSchedulerDescriptor.java | 32 ++++
tez-api/src/main/proto/DAGApiRecords.proto | 25 +++
.../apache/tez/client/TestTezClientUtils.java | 12 +-
.../org/apache/tez/common/TezUtilsInternal.java | 31 +++-
.../java/org/apache/tez/client/LocalClient.java | 2 +-
.../org/apache/tez/dag/app/DAGAppMaster.java | 172 +++++++++++--------
.../dag/app/TaskAttemptListenerImpTezDag.java | 37 ++--
.../java/org/apache/tez/dag/app/dag/DAG.java | 2 +
.../apache/tez/dag/app/dag/impl/DAGImpl.java | 9 +
.../apache/tez/dag/app/dag/impl/VertexImpl.java | 39 ++++-
.../dag/app/launcher/ContainerLauncherImpl.java | 6 +-
.../app/launcher/ContainerLauncherRouter.java | 40 +++--
.../dag/app/rm/TaskSchedulerEventHandler.java | 47 ++---
.../apache/tez/dag/app/MockDAGAppMaster.java | 5 +-
.../app/TestTaskAttemptListenerImplTezDag.java | 5 +-
.../app/rm/TestTaskSchedulerEventHandler.java | 4 +-
.../dag/app/rm/TestTaskSchedulerHelpers.java | 3 +-
.../org/apache/tez/examples/JoinValidate.java | 31 ++--
.../TezTestServiceContainerLauncher.java | 3 +-
.../tez/examples/JoinValidateConfigured.java | 40 +++--
.../tez/tests/TestExternalTezServices.java | 131 +++++++-------
.../org/apache/tez/runtime/task/TezChild.java | 5 +-
34 files changed, 995 insertions(+), 311 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index e57f76f..a201942 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -37,5 +37,6 @@ ALL CHANGES:
TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
TEZ-2005. Define basic interface for pluggable TaskScheduler.
TEZ-2651. Pluggable services should not extend AbstractService.
+ TEZ-2652. Cleanup the way services are specified for an AM and vertices.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index aad6e76..f961291 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -25,6 +25,7 @@ import java.util.Map;
import javax.annotation.Nullable;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -110,8 +111,9 @@ public class TezClient {
private static final long SLEEP_FOR_READY = 500;
private JobTokenSecretManager jobTokenSecretManager =
new JobTokenSecretManager();
- private Map<String, LocalResource> additionalLocalResources = Maps.newHashMap();
- private TezApiVersionInfo apiVersionInfo;
+ private final Map<String, LocalResource> additionalLocalResources = Maps.newHashMap();
+ private final TezApiVersionInfo apiVersionInfo;
+ private final ServicePluginsDescriptor servicePluginsDescriptor;
private HistoryACLPolicyManager historyACLPolicyManager;
private int preWarmDAGCounter = 0;
@@ -141,18 +143,44 @@ public class TezClient {
@Private
protected TezClient(String name, TezConfiguration tezConf, boolean isSession,
+ @Nullable Map<String, LocalResource> localResources,
+ @Nullable Credentials credentials) {
+ this(name, tezConf, isSession, localResources, credentials, null);
+ }
+
+ @Private
+ protected TezClient(String name, TezConfiguration tezConf, boolean isSession,
@Nullable Map<String, LocalResource> localResources,
- @Nullable Credentials credentials) {
+ @Nullable Credentials credentials, ServicePluginsDescriptor servicePluginsDescriptor) {
this.clientName = name;
this.isSession = isSession;
// Set in conf for local mode AM to figure out whether in session mode or not
tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, isSession);
this.amConfig = new AMConfiguration(tezConf, localResources, credentials);
this.apiVersionInfo = new TezApiVersionInfo();
+ this.servicePluginsDescriptor = servicePluginsDescriptor;
LOG.info("Tez Client Version: " + apiVersionInfo.toString());
}
+
+ /**
+ * Create a new TezClientBuilder. This can be used to setup additional parameters
+ * like session mode, local resources, credentials, servicePlugins, etc.
+ * <p/>
+ * If session mode is not specified in the builder, this will be inferred from
+ * the provided TezConfiguration.
+ *
+ * @param name Name of the client. Used for logging etc. This will also be used
+ * as app master name is session mode
+ * @param tezConf Configuration for the framework
+ * @return An instance of {@link org.apache.tez.client.TezClient.TezClientBuilder}
+ * which can be used to construct the final TezClient.
+ */
+ public static TezClientBuilder newBuilder(String name, TezConfiguration tezConf) {
+ return new TezClientBuilder(name, tezConf);
+ }
+
/**
* Create a new TezClient. Session or non-session execution mode will be
* inferred from configuration.
@@ -354,7 +382,7 @@ public class TezClient {
sessionAppId,
null, clientName, amConfig,
tezJarResources, sessionCredentials, usingTezArchiveDeploy, apiVersionInfo,
- historyACLPolicyManager);
+ historyACLPolicyManager, servicePluginsDescriptor);
// Set Tez Sessions to not retry on AM crashes if recovery is disabled
if (!amConfig.getTezConfiguration().getBoolean(
@@ -770,7 +798,8 @@ public class TezClient {
ApplicationSubmissionContext appContext = TezClientUtils
.createApplicationSubmissionContext(
appId, dag, dag.getName(), amConfig, tezJarResources, credentials,
- usingTezArchiveDeploy, apiVersionInfo, historyACLPolicyManager);
+ usingTezArchiveDeploy, apiVersionInfo, historyACLPolicyManager,
+ servicePluginsDescriptor);
LOG.info("Submitting DAG to YARN"
+ ", applicationId=" + appId
+ ", dagName=" + dag.getName());
@@ -845,4 +874,46 @@ public class TezClient {
append(SEPARATOR).
append(tezDagIdFormat.get().format(1)).toString();
}
+
+ @Public
+ public static class TezClientBuilder {
+ final String name;
+ final TezConfiguration tezConf;
+ boolean isSession;
+ private Map<String, LocalResource> localResourceMap;
+ private Credentials credentials;
+ ServicePluginsDescriptor servicePluginsDescriptor;
+
+ private TezClientBuilder(String name, TezConfiguration tezConf) {
+ this.name = name;
+ this.tezConf = tezConf;
+ isSession = tezConf.getBoolean(
+ TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT);
+ }
+
+ public TezClientBuilder setIsSession(boolean isSession) {
+ this.isSession = isSession;
+ return this;
+ }
+
+ public TezClientBuilder setLocalResources(Map<String, LocalResource> localResources) {
+ this.localResourceMap = localResources;
+ return this;
+ }
+
+ public TezClientBuilder setCredentials(Credentials credentials) {
+ this.credentials = credentials;
+ return this;
+ }
+
+ public TezClientBuilder setServicePluginDescriptor(ServicePluginsDescriptor servicePluginsDescriptor) {
+ this.servicePluginsDescriptor = servicePluginsDescriptor;
+ return this;
+ }
+
+ public TezClient build() {
+ return new TezClient(name, tezConf, isSession, localResourceMap, credentials,
+ servicePluginsDescriptor);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index 8bfaa1f..9cf1f3f 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -39,6 +39,10 @@ import java.util.Map.Entry;
import com.google.common.base.Strings;
import org.apache.commons.lang.StringUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -405,6 +409,7 @@ public class TezClientUtils {
* @param tezJarResources Resources to be used by the AM
* @param sessionCreds the credential object which will be populated with session specific
* @param historyACLPolicyManager
+ * @param servicePluginsDescriptor descriptor for services which may be running in the AM
* @return an ApplicationSubmissionContext to launch a Tez AM
* @throws IOException
* @throws YarnException
@@ -415,7 +420,8 @@ public class TezClientUtils {
ApplicationId appId, DAG dag, String amName,
AMConfiguration amConfig, Map<String, LocalResource> tezJarResources,
Credentials sessionCreds, boolean tezLrsAsArchive,
- TezApiVersionInfo apiVersionInfo, HistoryACLPolicyManager historyACLPolicyManager)
+ TezApiVersionInfo apiVersionInfo, HistoryACLPolicyManager historyACLPolicyManager,
+ ServicePluginsDescriptor servicePluginsDescriptor)
throws IOException, YarnException {
Preconditions.checkNotNull(sessionCreds);
@@ -551,7 +557,7 @@ public class TezClientUtils {
// emit conf as PB file
ConfigurationProto finalConfProto = createFinalConfProtoForApp(amConfig.getTezConfiguration(),
- aclConfigs);
+ aclConfigs, servicePluginsDescriptor);
FSDataOutputStream amConfPBOutBinaryStream = null;
try {
@@ -752,12 +758,8 @@ public class TezClientUtils {
+ "," + TezConstants.TEZ_CONTAINER_LOGGER_NAME);
}
- static ConfigurationProto createFinalConfProtoForApp(Configuration amConf) {
- return createFinalConfProtoForApp(amConf, null);
- }
-
static ConfigurationProto createFinalConfProtoForApp(Configuration amConf,
- Map<String, String> additionalConfigs) {
+ Map<String, String> additionalConfigs, ServicePluginsDescriptor servicePluginsDescriptor) {
assert amConf != null;
ConfigurationProto.Builder builder = ConfigurationProto.newBuilder();
for (Entry<String, String> entry : amConf) {
@@ -774,9 +776,49 @@ public class TezClientUtils {
builder.addConfKeyValues(kvp);
}
}
+
+ AMPluginDescriptorProto pluginDescriptorProto =
+ createAMServicePluginDescriptorProto(servicePluginsDescriptor);
+ builder.setAmPluginDescriptor(pluginDescriptorProto);
+
return builder.build();
}
+ static AMPluginDescriptorProto createAMServicePluginDescriptorProto(
+ ServicePluginsDescriptor servicePluginsDescriptor) {
+ AMPluginDescriptorProto.Builder pluginDescriptorBuilder =
+ AMPluginDescriptorProto.newBuilder();
+ if (servicePluginsDescriptor != null) {
+
+ pluginDescriptorBuilder.setContainersEnabled(servicePluginsDescriptor.areContainersEnabled());
+ pluginDescriptorBuilder.setUberEnabled(servicePluginsDescriptor.isUberEnabled());
+
+ if (servicePluginsDescriptor.getTaskSchedulerDescriptors() != null &&
+ servicePluginsDescriptor.getTaskSchedulerDescriptors().length > 0) {
+ List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+ servicePluginsDescriptor.getTaskSchedulerDescriptors());
+ pluginDescriptorBuilder.addAllTaskScedulers(namedEntityProtos);
+ }
+
+ if (servicePluginsDescriptor.getContainerLauncherDescriptors() != null &&
+ servicePluginsDescriptor.getContainerLauncherDescriptors().length > 0) {
+ List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+ servicePluginsDescriptor.getContainerLauncherDescriptors());
+ pluginDescriptorBuilder.addAllContainerLaunchers(namedEntityProtos);
+ }
+
+ if (servicePluginsDescriptor.getTaskCommunicatorDescriptors() != null &&
+ servicePluginsDescriptor.getTaskCommunicatorDescriptors().length > 0) {
+ List<TezNamedEntityDescriptorProto> namedEntityProtos = DagTypeConverters.convertNamedEntityCollectionToProto(
+ servicePluginsDescriptor.getTaskCommunicatorDescriptors());
+ pluginDescriptorBuilder.addAllTaskCommunicators(namedEntityProtos);
+ }
+
+ } else {
+ pluginDescriptorBuilder.setContainersEnabled(true).setUberEnabled(false);
+ }
+ return pluginDescriptorBuilder.build();
+ }
/**
* Helper function to create a YARN LocalResource
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
index 8ee1682..fce9522 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
@@ -32,6 +32,9 @@ import java.util.Stack;
import org.apache.commons.collections4.BidiMap;
import org.apache.commons.collections4.bidimap.DualLinkedHashBidiMap;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
+import org.apache.tez.dag.api.records.DAGProtos;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -60,9 +63,7 @@ import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
import com.google.common.base.Preconditions;
-import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
@@ -92,6 +93,7 @@ public class DAG {
Map<String, LocalResource> commonTaskLocalFiles = Maps.newHashMap();
String dagInfo;
private Map<String,String> dagConf = new HashMap<String, String>();
+ private VertexExecutionContext defaultExecutionContext;
private Stack<String> topologicalVertexStack = new Stack<String>();
@@ -335,6 +337,26 @@ public class DAG {
return this;
}
+ /**
+ * Sets the default execution context for the DAG. This can be overridden at a per Vertex level.
+ * See {@link org.apache.tez.dag.api.Vertex#setExecutionContext(VertexExecutionContext)}
+ *
+ * @param vertexExecutionContext the default execution context for the DAG
+ *
+ * @return
+ */
+ @Public
+ @InterfaceStability.Unstable
+ public synchronized DAG setExecutionContext(VertexExecutionContext vertexExecutionContext) {
+ this.defaultExecutionContext = vertexExecutionContext;
+ return this;
+ }
+
+ @Private
+ VertexExecutionContext getDefaultExecutionContext() {
+ return this.defaultExecutionContext;
+ }
+
@Private
@VisibleForTesting
public Map<String,String> getDagConf() {
@@ -707,7 +729,15 @@ public class DAG {
if (this.dagInfo != null && !this.dagInfo.isEmpty()) {
dagBuilder.setDagInfo(this.dagInfo);
}
-
+
+ // Setup default execution context.
+ VertexExecutionContext defaultContext = getDefaultExecutionContext();
+ if (defaultContext != null) {
+ DAGProtos.VertexExecutionContextProto contextProto = DagTypeConverters.convertToProto(
+ defaultContext);
+ dagBuilder.setDefaultExecutionContext(contextProto);
+ }
+
if (!vertexGroups.isEmpty()) {
for (VertexGroup av : vertexGroups) {
GroupInfo groupInfo = av.getGroupInfo();
@@ -800,7 +830,17 @@ public class DAG {
vertexBuilder.setName(vertex.getName());
vertexBuilder.setType(PlanVertexType.NORMAL); // vertex type is implicitly NORMAL until TEZ-46.
vertexBuilder.setProcessorDescriptor(DagTypeConverters
- .convertToDAGPlan(vertex.getProcessorDescriptor()));
+ .convertToDAGPlan(vertex.getProcessorDescriptor()));
+
+ // Vertex ExecutionContext setup
+ VertexExecutionContext execContext = vertex.getVertexExecutionContext();
+ if (execContext != null) {
+ DAGProtos.VertexExecutionContextProto contextProto =
+ DagTypeConverters.convertToProto(execContext);
+ vertexBuilder.setExecutionContext(contextProto);
+ }
+ // End of VertexExecutionContext setup.
+
if (vertex.getInputs().size() > 0) {
for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input : vertex.getInputs()) {
vertexBuilder.addInputs(DagTypeConverters.convertToDAGPlan(input));
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
index 8b1d553..2e0d417 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
@@ -73,11 +73,15 @@ import org.apache.tez.dag.api.records.DAGProtos.TezCounterGroupProto;
import org.apache.tez.dag.api.records.DAGProtos.TezCounterProto;
import org.apache.tez.dag.api.records.DAGProtos.TezCountersProto;
import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
import org.apache.tez.dag.api.records.DAGProtos.VertexLocationHintProto;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
import com.google.protobuf.ByteString.Output;
+import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
+import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
+import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
@Private
public class DagTypeConverters {
@@ -399,6 +403,8 @@ public class DagTypeConverters {
return userPayload;
}
+
+
private static void setUserPayload(EntityDescriptor<?> entity, UserPayload payload) {
if (payload != null) {
entity.setUserPayload(payload);
@@ -423,6 +429,15 @@ public class DagTypeConverters {
return od;
}
+ public static NamedEntityDescriptor convertNamedDescriptorFromProto(TezNamedEntityDescriptorProto proto) {
+ String name = proto.getName();
+ String className = proto.getEntityDescriptor().getClassName();
+ UserPayload payload = convertTezUserPayloadFromDAGPlan(proto.getEntityDescriptor());
+ NamedEntityDescriptor descriptor = new NamedEntityDescriptor(name, className);
+ setUserPayload(descriptor, payload);
+ return descriptor;
+ }
+
public static InputInitializerDescriptor convertInputInitializerDescriptorFromDAGPlan(
TezEntityDescriptorProto proto) {
String className = proto.getClassName();
@@ -550,11 +565,11 @@ public class DagTypeConverters {
public static LocalResource convertPlanLocalResourceToLocalResource(
PlanLocalResource plr) {
return LocalResource.newInstance(
- ConverterUtils.getYarnUrlFromPath(new Path(plr.getUri())),
- DagTypeConverters.convertFromDAGPlan(plr.getType()),
- DagTypeConverters.convertFromDAGPlan(plr.getVisibility()),
- plr.getSize(), plr.getTimeStamp(),
- plr.hasPattern() ? plr.getPattern() : null);
+ ConverterUtils.getYarnUrlFromPath(new Path(plr.getUri())),
+ DagTypeConverters.convertFromDAGPlan(plr.getType()),
+ DagTypeConverters.convertFromDAGPlan(plr.getVisibility()),
+ plr.getSize(), plr.getTimeStamp(),
+ plr.hasPattern() ? plr.getPattern() : null);
}
public static TezCounters convertTezCountersFromProto(TezCountersProto proto) {
@@ -717,4 +732,72 @@ public class DagTypeConverters {
return payload.getPayload();
}
+ public static DAGProtos.VertexExecutionContextProto convertToProto(
+ Vertex.VertexExecutionContext context) {
+ if (context == null) {
+ return null;
+ } else {
+ DAGProtos.VertexExecutionContextProto.Builder builder =
+ DAGProtos.VertexExecutionContextProto.newBuilder();
+ builder.setExecuteInAm(context.shouldExecuteInAm());
+ builder.setExecuteInContainers(context.shouldExecuteInContainers());
+ if (context.getTaskSchedulerName() != null) {
+ builder.setTaskSchedulerName(context.getTaskSchedulerName());
+ }
+ if (context.getContainerLauncherName() != null) {
+ builder.setContainerLauncherName(context.getContainerLauncherName());
+ }
+ if (context.getTaskCommName() != null) {
+ builder.setTaskCommName(context.getTaskCommName());
+ }
+ return builder.build();
+ }
+ }
+
+ public static Vertex.VertexExecutionContext convertFromProto(
+ DAGProtos.VertexExecutionContextProto proto) {
+ if (proto == null) {
+ return null;
+ } else {
+ if (proto.getExecuteInAm()) {
+ Vertex.VertexExecutionContext context =
+ Vertex.VertexExecutionContext.createExecuteInAm(proto.getExecuteInAm());
+ return context;
+ } else if (proto.getExecuteInContainers()) {
+ Vertex.VertexExecutionContext context =
+ Vertex.VertexExecutionContext.createExecuteInContainers(proto.getExecuteInContainers());
+ return context;
+ } else {
+ String taskScheduler = proto.hasTaskSchedulerName() ? proto.getTaskSchedulerName() : null;
+ String containerLauncher =
+ proto.hasContainerLauncherName() ? proto.getContainerLauncherName() : null;
+ String taskComm = proto.hasTaskCommName() ? proto.getTaskCommName() : null;
+ Vertex.VertexExecutionContext context =
+ Vertex.VertexExecutionContext.create(taskScheduler, containerLauncher, taskComm);
+ return context;
+ }
+ }
+ }
+
+ public static List<TezNamedEntityDescriptorProto> convertNamedEntityCollectionToProto(
+ NamedEntityDescriptor[] namedEntityDescriptors) {
+ List<TezNamedEntityDescriptorProto> list =
+ Lists.newArrayListWithCapacity(namedEntityDescriptors.length);
+ for (NamedEntityDescriptor namedEntity : namedEntityDescriptors) {
+ TezNamedEntityDescriptorProto namedEntityProto = convertNamedEntityToProto(namedEntity);
+ list.add(namedEntityProto);
+ }
+ return list;
+ }
+
+ public static TezNamedEntityDescriptorProto convertNamedEntityToProto(
+ NamedEntityDescriptor namedEntityDescriptor) {
+ TezNamedEntityDescriptorProto.Builder builder = TezNamedEntityDescriptorProto.newBuilder();
+ builder.setName(namedEntityDescriptor.getEntityName());
+ DAGProtos.TezEntityDescriptorProto entityProto =
+ DagTypeConverters.convertToDAGPlan(namedEntityDescriptor);
+ builder.setEntityDescriptor(entityProto);
+ return builder.build();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
new file mode 100644
index 0000000..bad0d10
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tez.dag.api;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+public class NamedEntityDescriptor<T extends EntityDescriptor<T>> extends EntityDescriptor<T> {
+ private final String entityName;
+
+ @InterfaceAudience.Private
+ public NamedEntityDescriptor(String entityName, String className) {
+ super(className);
+ Preconditions.checkArgument(entityName != null, "EntityName must be specified");
+ this.entityName = entityName;
+ }
+
+ public String getEntityName() {
+ return entityName;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 39a4c77..3b7378a 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -1215,37 +1215,6 @@ public class TezConfiguration extends Configuration {
+ "tez-ui.webservice.enable";
public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true;
- /** defaults container-launcher for the specific vertex */
- @ConfigurationScope(Scope.VERTEX)
- public static final String TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME = TEZ_AM_PREFIX + "vertex.container-launcher.name";
- /** defaults task-scheduler for the specific vertex */
- @ConfigurationScope(Scope.VERTEX)
- public static final String TEZ_AM_VERTEX_TASK_SCHEDULER_NAME = TEZ_AM_PREFIX + "vertex.task-scheduler.name";
- /** defaults task-communicator for the specific vertex */
- @ConfigurationScope(Scope.VERTEX)
- public static final String TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME = TEZ_AM_PREFIX + "vertex.task-communicator.name";
-
- /** Comma separated list of named container-launcher classes running in the AM.
- * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
- * e.g. Tez, ExtService:org.apache.ExtLauncherClasss
- * */
- @ConfigurationScope(Scope.AM)
- public static final String TEZ_AM_CONTAINER_LAUNCHERS = TEZ_AM_PREFIX + "container-launchers";
-
- /** Comma separated list of task-schedulers classes running in the AM.
- * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
- * e.g. Tez, ExtService:org.apache.ExtSchedulerClasss
- */
- @ConfigurationScope(Scope.AM)
- public static final String TEZ_AM_TASK_SCHEDULERS = TEZ_AM_PREFIX + "task-schedulers";
-
- /** Comma separated list of task-communicators classes running in the AM.
- * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
- * e.g. Tez, ExtService:org.apache.ExtTaskCommClass
- * */
- @ConfigurationScope(Scope.AM)
- public static final String TEZ_AM_TASK_COMMUNICATORS = TEZ_AM_PREFIX + "task-communicators";
-
// TODO only validate property here, value can also be validated if necessary
public static void validateProperty(String property, Scope usedScope) {
Scope validScope = PropertyScope.get(property);
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
index 3b07c59..6e1cb2d 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
@@ -102,7 +102,14 @@ public class TezConstants {
/// Version-related Environment variables
public static final String TEZ_CLIENT_VERSION_ENV = "TEZ_CLIENT_VERSION";
+ private static final String TEZ_AM_SERVICE_PLUGIN_NAME_YARN_CONTAINERS = "TezYarn";
+ private static final String TEZ_AM_SERVICE_PLUGIN_NAME_IN_AM = "TezUber";
- public static final String TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT = "Tez";
- public static final String TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT = "TezLocal";
+ public static String getTezYarnServicePluginName() {
+ return TEZ_AM_SERVICE_PLUGIN_NAME_YARN_CONTAINERS;
+ }
+
+ public static String getTezUberServicePluginName() {
+ return TEZ_AM_SERVICE_PLUGIN_NAME_IN_AM;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
index 0ed4bd8..34124b2 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
@@ -28,11 +28,11 @@ import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.dag.api.VertexGroup.GroupInfo;
-import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.runtime.api.LogicalIOProcessor;
import com.google.common.base.Preconditions;
@@ -57,6 +57,7 @@ public class Vertex {
private final Map<String, LocalResource> taskLocalResources = new HashMap<String, LocalResource>();
private Map<String, String> taskEnvironment = new HashMap<String, String>();
private Map<String, String> vertexConf = new HashMap<String, String>();
+ private VertexExecutionContext vertexExecutionContext;
private final Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> additionalInputs
= new HashMap<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>();
private final Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> additionalOutputs
@@ -410,6 +411,108 @@ public class Vertex {
return this;
}
+ /**
+ * Sets the execution context for this Vertex - i.e. the Task Scheduler, ContainerLauncher and
+ * TaskCommunicator to be used. Also whether the vertex will be executed within the AM.
+ * If partially specified, the default components in Tez will be used - which may or may not work
+ * with the custom context.
+ *
+ * @param vertexExecutionContext the execution context for the vertex.
+ *
+ * @return
+ */
+ public Vertex setExecutionContext(VertexExecutionContext vertexExecutionContext) {
+ this.vertexExecutionContext = vertexExecutionContext;
+ return this;
+ }
+
+ @Public
+ @InterfaceStability.Unstable
+ public static class VertexExecutionContext {
+ final boolean executeInAm;
+ final boolean executeInContainers;
+ final String taskSchedulerName;
+ final String containerLauncherName;
+ final String taskCommName;
+
+ public static VertexExecutionContext createExecuteInAm(boolean executeInAm) {
+ return new VertexExecutionContext(executeInAm, false);
+ }
+
+ public static VertexExecutionContext createExecuteInContainers(boolean executeInContainers) {
+ return new VertexExecutionContext(false, executeInContainers);
+ }
+
+ public static VertexExecutionContext create(String taskSchedulerName, String containerLauncherName,
+ String taskCommName) {
+ return new VertexExecutionContext(taskSchedulerName, containerLauncherName, taskCommName);
+ }
+
+ private VertexExecutionContext(boolean executeInAm, boolean executeInContainers) {
+ this(executeInAm, executeInContainers, null, null, null);
+ }
+
+ private VertexExecutionContext(String taskSchedulerName, String containerLauncherName,
+ String taskCommName) {
+ this(false, false, taskSchedulerName, containerLauncherName, taskCommName);
+ }
+
+ private VertexExecutionContext(boolean executeInAm, boolean executeInContainers, String taskSchedulerName, String containerLauncherName,
+ String taskCommName) {
+ if (executeInAm || executeInContainers) {
+ Preconditions.checkState(!(executeInAm && executeInContainers),
+ "executeInContainers and executeInAM are mutually exclusive");
+ Preconditions.checkState(
+ taskSchedulerName == null && containerLauncherName == null && taskCommName == null,
+ "Uber (in-AM) or container execution cannot be enabled with a custom plugins. TaskScheduler=" +
+ taskSchedulerName + ", ContainerLauncher=" + containerLauncherName +
+ ", TaskCommunicator=" + taskCommName);
+ }
+ if (taskSchedulerName != null || containerLauncherName != null || taskCommName != null) {
+ Preconditions.checkState(executeInAm == false && executeInContainers == false,
+ "Uber (in-AM) and container execution cannot be enabled with a custom plugins. TaskScheduler=" +
+ taskSchedulerName + ", ContainerLauncher=" + containerLauncherName +
+ ", TaskCommunicator=" + taskCommName);
+ }
+ this.executeInAm = executeInAm;
+ this.executeInContainers = executeInContainers;
+ this.taskSchedulerName = taskSchedulerName;
+ this.containerLauncherName = containerLauncherName;
+ this.taskCommName = taskCommName;
+ }
+
+ public boolean shouldExecuteInAm() {
+ return executeInAm;
+ }
+
+ public boolean shouldExecuteInContainers() {
+ return executeInContainers;
+ }
+
+ public String getTaskSchedulerName() {
+ return taskSchedulerName;
+ }
+
+ public String getContainerLauncherName() {
+ return containerLauncherName;
+ }
+
+ public String getTaskCommName() {
+ return taskCommName;
+ }
+
+ @Override
+ public String toString() {
+ return "VertexExecutionContext{" +
+ "executeInAm=" + executeInAm +
+ ", executeInContainers=" + executeInContainers +
+ ", taskSchedulerName='" + taskSchedulerName + '\'' +
+ ", containerLauncherName='" + containerLauncherName + '\'' +
+ ", taskCommName='" + taskCommName + '\'' +
+ '}';
+ }
+ }
+
@Override
public String toString() {
return "[" + vertexName + " : " + processorDescriptor.getClassName() + "]";
@@ -475,6 +578,11 @@ public class Vertex {
return dataSinks;
}
+ @Private
+ public VertexExecutionContext getVertexExecutionContext() {
+ return this.vertexExecutionContext;
+ }
+
List<Edge> getInputEdges() {
return inputEdges;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherDescriptor.java
new file mode 100644
index 0000000..ff3c90e
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherDescriptor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerLauncherDescriptor extends NamedEntityDescriptor<ContainerLauncherDescriptor> {
+
+ private ContainerLauncherDescriptor(String containerLauncherName, String containerLauncherClassname) {
+ super(containerLauncherName, containerLauncherClassname);
+ }
+
+ public static ContainerLauncherDescriptor create(String containerLauncherName, String containerLauncherClassname) {
+ return new ContainerLauncherDescriptor(containerLauncherName, containerLauncherClassname);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
new file mode 100644
index 0000000..8df102a
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ServicePluginsDescriptor.java
@@ -0,0 +1,96 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ServicePluginsDescriptor {
+
+ private final boolean enableContainers;
+ private final boolean enableUber;
+
+ private TaskSchedulerDescriptor[] taskSchedulerDescriptors;
+ private ContainerLauncherDescriptor[] containerLauncherDescriptors;
+ private TaskCommunicatorDescriptor[] taskCommunicatorDescriptors;
+
+ private ServicePluginsDescriptor(boolean enableContainers, boolean enableUber,
+ TaskSchedulerDescriptor[] taskSchedulerDescriptor,
+ ContainerLauncherDescriptor[] containerLauncherDescriptors,
+ TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
+ this.enableContainers = enableContainers;
+ this.enableUber = enableUber;
+ Preconditions.checkArgument(taskSchedulerDescriptors == null || taskSchedulerDescriptors.length > 0,
+ "TaskSchedulerDescriptors should either not be specified or at least 1 should be provided");
+ this.taskSchedulerDescriptors = taskSchedulerDescriptor;
+ Preconditions.checkArgument(containerLauncherDescriptors == null || containerLauncherDescriptors.length > 0,
+ "ContainerLauncherDescriptor should either not be specified or at least 1 should be provided");
+ this.containerLauncherDescriptors = containerLauncherDescriptors;
+ Preconditions.checkArgument(taskCommunicatorDescriptors == null || taskCommunicatorDescriptors.length > 0,
+ "TaskCommunicatorDescriptors should either not be specified or at least 1 should be provided");
+ this.taskCommunicatorDescriptors = taskCommunicatorDescriptors;
+ }
+
+ public static ServicePluginsDescriptor create(TaskSchedulerDescriptor[] taskSchedulerDescriptor,
+ ContainerLauncherDescriptor[] containerLauncherDescriptors,
+ TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
+ return new ServicePluginsDescriptor(true, false, taskSchedulerDescriptor,
+ containerLauncherDescriptors, taskCommunicatorDescriptors);
+ }
+
+ public static ServicePluginsDescriptor create(boolean enableUber,
+ TaskSchedulerDescriptor[] taskSchedulerDescriptor,
+ ContainerLauncherDescriptor[] containerLauncherDescriptors,
+ TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
+ return new ServicePluginsDescriptor(true, enableUber, taskSchedulerDescriptor,
+ containerLauncherDescriptors, taskCommunicatorDescriptors);
+ }
+
+ public static ServicePluginsDescriptor create(boolean enableContainers, boolean enableUber,
+ TaskSchedulerDescriptor[] taskSchedulerDescriptor,
+ ContainerLauncherDescriptor[] containerLauncherDescriptors,
+ TaskCommunicatorDescriptor[] taskCommunicatorDescriptors) {
+ return new ServicePluginsDescriptor(enableContainers, enableUber, taskSchedulerDescriptor,
+ containerLauncherDescriptors, taskCommunicatorDescriptors);
+ }
+
+ public static ServicePluginsDescriptor create(boolean enableUber) {
+ return new ServicePluginsDescriptor(true, enableUber, null, null, null);
+ }
+
+
+ public boolean areContainersEnabled() {
+ return enableContainers;
+ }
+
+ public boolean isUberEnabled() {
+ return enableUber;
+ }
+
+ public TaskSchedulerDescriptor[] getTaskSchedulerDescriptors() {
+ return taskSchedulerDescriptors;
+ }
+
+ public ContainerLauncherDescriptor[] getContainerLauncherDescriptors() {
+ return containerLauncherDescriptors;
+ }
+
+ public TaskCommunicatorDescriptor[] getTaskCommunicatorDescriptors() {
+ return taskCommunicatorDescriptors;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorDescriptor.java
new file mode 100644
index 0000000..57ac385
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorDescriptor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TaskCommunicatorDescriptor extends NamedEntityDescriptor<TaskCommunicatorDescriptor> {
+
+
+ private TaskCommunicatorDescriptor(String taskCommName, String taskCommClassname) {
+ super(taskCommName, taskCommClassname);
+ }
+
+ public static TaskCommunicatorDescriptor create(String taskCommName, String taskCommClassname) {
+ return new TaskCommunicatorDescriptor(taskCommName, taskCommClassname);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerDescriptor.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerDescriptor.java
new file mode 100644
index 0000000..12e0919
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerDescriptor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TaskSchedulerDescriptor extends NamedEntityDescriptor<TaskSchedulerDescriptor> {
+
+ private TaskSchedulerDescriptor(String taskSchedulerName, String schedulerClassname) {
+ super(taskSchedulerName, schedulerClassname);
+ }
+
+ public static TaskSchedulerDescriptor create(String taskSchedulerName, String schedulerClassName) {
+ return new TaskSchedulerDescriptor(taskSchedulerName, schedulerClassName);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/main/proto/DAGApiRecords.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto
index 959d4e6..ebe3259 100644
--- a/tez-api/src/main/proto/DAGApiRecords.proto
+++ b/tez-api/src/main/proto/DAGApiRecords.proto
@@ -127,6 +127,14 @@ message RootInputLeafOutputProto {
optional TezEntityDescriptorProto controller_descriptor = 3;
}
+message VertexExecutionContextProto {
+ optional bool execute_in_am = 1;
+ optional bool execute_in_containers = 2;
+ optional string task_scheduler_name = 3;
+ optional string container_launcher_name = 4;
+ optional string task_comm_name = 5;
+}
+
message VertexPlan {
required string name = 1;
required PlanVertexType type = 2;
@@ -139,6 +147,7 @@ message VertexPlan {
repeated RootInputLeafOutputProto outputs = 9;
optional TezEntityDescriptorProto vertex_manager_plugin = 10;
optional ConfigurationProto vertexConf = 11;
+ optional VertexExecutionContextProto execution_context = 12;
}
message PlanEdgeProperty {
@@ -162,8 +171,23 @@ message EdgePlan {
optional TezEntityDescriptorProto edge_manager = 9;
}
+message TezNamedEntityDescriptorProto {
+ optional string name = 1;
+ optional TezEntityDescriptorProto entity_descriptor = 2;
+}
+
+
+message AMPluginDescriptorProto {
+ optional bool containers_enabled = 1 [default = true];
+ optional bool uber_enabled = 2 [default = false];
+ repeated TezNamedEntityDescriptorProto task_scedulers = 3;
+ repeated TezNamedEntityDescriptorProto container_launchers = 4;
+ repeated TezNamedEntityDescriptorProto task_communicators = 5;
+}
+
message ConfigurationProto {
repeated PlanKeyValuePair confKeyValues = 1;
+ optional AMPluginDescriptorProto am_plugin_descriptor = 2;
}
message DAGPlan {
@@ -175,6 +199,7 @@ message DAGPlan {
repeated PlanVertexGroupInfo vertex_groups = 6;
repeated PlanLocalResource local_resource = 7;
optional string dag_info = 8;
+ optional VertexExecutionContextProto default_execution_context = 9;
}
// DAG monitoring messages
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
index 2d4e005..8946ef0 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
@@ -261,7 +261,7 @@ public class TestTezClientUtils {
ApplicationSubmissionContext appSubmissionContext =
TezClientUtils.createApplicationSubmissionContext(appId, dag, "amName", amConf,
new HashMap<String, LocalResource>(), credentials, false, new TezApiVersionInfo(),
- mock(HistoryACLPolicyManager.class));
+ mock(HistoryACLPolicyManager.class), null);
ContainerLaunchContext amClc = appSubmissionContext.getAMContainerSpec();
Map<String, ByteBuffer> amServiceData = amClc.getServiceData();
@@ -294,7 +294,7 @@ public class TestTezClientUtils {
ApplicationSubmissionContext appSubmissionContext =
TezClientUtils.createApplicationSubmissionContext(appId, dag, "amName", amConf,
new HashMap<String, LocalResource>(), credentials, false, new TezApiVersionInfo(),
- mock(HistoryACLPolicyManager.class));
+ mock(HistoryACLPolicyManager.class), null);
List<String> expectedCommands = new LinkedList<String>();
expectedCommands.add("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator");
@@ -334,7 +334,7 @@ public class TestTezClientUtils {
ApplicationSubmissionContext appSubmissionContext =
TezClientUtils.createApplicationSubmissionContext(appId, dag, "amName", amConf,
new HashMap<String, LocalResource>(), credentials, false, new TezApiVersionInfo(),
- mock(HistoryACLPolicyManager.class));
+ mock(HistoryACLPolicyManager.class), null);
List<String> expectedCommands = new LinkedList<String>();
expectedCommands.add("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator");
@@ -516,7 +516,7 @@ public class TestTezClientUtils {
expected.put("property1", val1);
expected.put("property2", expVal2);
- ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(conf, null);
+ ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(conf, null, null);
for (PlanKeyValuePair kvPair : confProto.getConfKeyValuesList()) {
String v = expected.remove(kvPair.getKey());
@@ -620,7 +620,7 @@ public class TestTezClientUtils {
srcConf.set(entry.getKey(), entry.getValue());
}
- ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(srcConf);
+ ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(srcConf, null, null);
for (PlanKeyValuePair kvPair : confProto.getConfKeyValuesList()) {
String val = confMap.remove(kvPair.getKey());
@@ -677,4 +677,6 @@ public class TestTezClientUtils {
Assert.assertTrue(resourceNames.contains("dir2-f.txt"));
}
+ // TODO TEZ-2003 Add test to validate ServicePluginDescriptor propagation
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
index 532e83c..1fb7ff9 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
@@ -57,7 +57,7 @@ public class TezUtilsInternal {
private static final Logger LOG = LoggerFactory.getLogger(TezUtilsInternal.class);
- public static void addUserSpecifiedTezConfiguration(String baseDir, Configuration conf) throws
+ public static ConfigurationProto readUserSpecifiedTezConfiguration(String baseDir) throws
IOException {
FileInputStream confPBBinaryStream = null;
ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder();
@@ -72,14 +72,41 @@ public class TezUtilsInternal {
}
ConfigurationProto confProto = confProtoBuilder.build();
+ return confProto;
+ }
- List<PlanKeyValuePair> kvPairList = confProto.getConfKeyValuesList();
+ public static void addUserSpecifiedTezConfiguration(Configuration conf,
+ List<PlanKeyValuePair> kvPairList) {
if (kvPairList != null && !kvPairList.isEmpty()) {
for (PlanKeyValuePair kvPair : kvPairList) {
conf.set(kvPair.getKey(), kvPair.getValue());
}
}
}
+//
+// public static void addUserSpecifiedTezConfiguration(String baseDir, Configuration conf) throws
+// IOException {
+// FileInputStream confPBBinaryStream = null;
+// ConfigurationProto.Builder confProtoBuilder = ConfigurationProto.newBuilder();
+// try {
+// confPBBinaryStream =
+// new FileInputStream(new File(baseDir, TezConstants.TEZ_PB_BINARY_CONF_NAME));
+// confProtoBuilder.mergeFrom(confPBBinaryStream);
+// } finally {
+// if (confPBBinaryStream != null) {
+// confPBBinaryStream.close();
+// }
+// }
+//
+// ConfigurationProto confProto = confProtoBuilder.build();
+//
+// List<PlanKeyValuePair> kvPairList = confProto.getConfKeyValuesList();
+// if (kvPairList != null && !kvPairList.isEmpty()) {
+// for (PlanKeyValuePair kvPair : kvPairList) {
+// conf.set(kvPair.getKey(), kvPair.getValue());
+// }
+// }
+// }
public static byte[] compressBytes(byte[] inBytes) throws IOException {
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
index 1bb2002..508f817 100644
--- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
+++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
@@ -341,7 +341,7 @@ public class LocalClient extends FrameworkClient {
String[] localDirs, String[] logDirs, Credentials credentials, String jobUserName) {
return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
new SystemClock(), appSubmitTime, isSession, userDir, localDirs, logDirs,
- versionInfo.getVersion(), 1, credentials, jobUserName);
+ versionInfo.getVersion(), 1, credentials, jobUserName, null);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index f3914d8..8388cfb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -40,6 +40,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
+import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@@ -61,7 +62,11 @@ import com.google.common.collect.HashBiMap;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.Options;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.SessionNotRunning;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
import org.apache.tez.dag.app.dag.event.DAGAppMasterEventDagCleanup;
import org.apache.tez.dag.history.events.DAGRecoveredEvent;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -221,6 +226,7 @@ public class DAGAppMaster extends AbstractService {
private final String workingDirectory;
private final String[] localDirs;
private final String[] logDirs;
+ private final AMPluginDescriptorProto amPluginDescriptorProto;
private ContainerSignatureMatcher containerSignatureMatcher;
private AMContainerMap containers;
private AMNodeTracker nodes;
@@ -312,7 +318,7 @@ public class DAGAppMaster extends AbstractService {
ContainerId containerId, String nmHost, int nmPort, int nmHttpPort,
Clock clock, long appSubmitTime, boolean isSession, String workingDirectory,
String [] localDirs, String[] logDirs, String clientVersion, int maxAppAttempts,
- Credentials credentials, String jobUserName) {
+ Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) {
super(DAGAppMaster.class.getName());
this.clock = clock;
this.startTime = clock.getTime();
@@ -332,6 +338,7 @@ public class DAGAppMaster extends AbstractService {
this.clientVersion = clientVersion;
this.maxAppAttempts = maxAppAttempts;
this.amCredentials = credentials;
+ this.amPluginDescriptorProto = pluginDescriptorProto;
this.appMasterUgi = UserGroupInformation
.createRemoteUser(jobUserName);
this.appMasterUgi.addCredentials(amCredentials);
@@ -380,28 +387,47 @@ public class DAGAppMaster extends AbstractService {
this.isLocal = conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
- String tezDefaultClassIdentifier =
- isLocal ? TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT :
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
+ List<NamedEntityDescriptor> taskSchedulerDescriptors;
+ List<NamedEntityDescriptor> containerLauncherDescriptors;
+ List<NamedEntityDescriptor> taskCommunicatorDescriptors;
+ boolean tezYarnEnabled = true;
+ boolean uberEnabled = false;
- String[] taskSchedulerClassIdentifiers = parsePlugins(taskSchedulers,
- conf.getTrimmedStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
- tezDefaultClassIdentifier),
- TezConfiguration.TEZ_AM_TASK_SCHEDULERS);
+ if (!isLocal) {
+ if (amPluginDescriptorProto == null) {
+ tezYarnEnabled = true;
+ uberEnabled = false;
+ } else {
+ tezYarnEnabled = amPluginDescriptorProto.getContainersEnabled();
+ uberEnabled = amPluginDescriptorProto.getUberEnabled();
+ }
+ } else {
+ tezYarnEnabled = false;
+ uberEnabled = true;
+ }
+
+ taskSchedulerDescriptors = parsePlugin(taskSchedulers,
+ (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskScedulersCount() == 0 ?
+ null :
+ amPluginDescriptorProto.getTaskScedulersList()),
+ tezYarnEnabled, uberEnabled);
- String[] containerLauncherClassIdentifiers = parsePlugins(containerLaunchers,
- conf.getTrimmedStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
- tezDefaultClassIdentifier),
- TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS);
+ containerLauncherDescriptors = parsePlugin(containerLaunchers,
+ (amPluginDescriptorProto == null ||
+ amPluginDescriptorProto.getContainerLaunchersCount() == 0 ? null :
+ amPluginDescriptorProto.getContainerLaunchersList()),
+ tezYarnEnabled, uberEnabled);
- String[] taskCommunicatorClassIdentifiers = parsePlugins(taskCommunicators,
- conf.getTrimmedStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
- tezDefaultClassIdentifier),
- TezConfiguration.TEZ_AM_TASK_COMMUNICATORS);
+ taskCommunicatorDescriptors = parsePlugin(taskCommunicators,
+ (amPluginDescriptorProto == null ||
+ amPluginDescriptorProto.getTaskCommunicatorsCount() == 0 ? null :
+ amPluginDescriptorProto.getTaskCommunicatorsList()),
+ tezYarnEnabled, uberEnabled);
- LOG.info(buildPluginComponentLog(taskSchedulerClassIdentifiers, taskSchedulers, "TaskSchedulers"));
- LOG.info(buildPluginComponentLog(containerLauncherClassIdentifiers, containerLaunchers, "ContainerLaunchers"));
- LOG.info(buildPluginComponentLog(taskCommunicatorClassIdentifiers, taskCommunicators, "TaskCommunicators"));
+
+ LOG.info(buildPluginComponentLog(taskSchedulerDescriptors, taskSchedulers, "TaskSchedulers"));
+ LOG.info(buildPluginComponentLog(containerLauncherDescriptors, containerLaunchers, "ContainerLaunchers"));
+ LOG.info(buildPluginComponentLog(taskCommunicatorDescriptors, taskCommunicators, "TaskCommunicators"));
boolean disableVersionCheck = conf.getBoolean(
TezConfiguration.TEZ_AM_DISABLE_CLIENT_VERSION_CHECK,
@@ -468,7 +494,7 @@ public class DAGAppMaster extends AbstractService {
//service to handle requests to TaskUmbilicalProtocol
taskAttemptListener = createTaskAttemptListener(context,
- taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorClassIdentifiers, isLocal);
+ taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors, isLocal);
addIfService(taskAttemptListener, true);
containerSignatureMatcher = createContainerSignatureMatcher();
@@ -516,7 +542,7 @@ public class DAGAppMaster extends AbstractService {
this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
- taskSchedulerClassIdentifiers, isLocal);
+ taskSchedulerDescriptors, isLocal);
addIfService(taskSchedulerEventHandler, true);
if (enableWebUIService()) {
@@ -534,7 +560,7 @@ public class DAGAppMaster extends AbstractService {
taskSchedulerEventHandler);
addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
- this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherClassIdentifiers, isLocal);
+ this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherDescriptors, isLocal);
addIfService(containerLauncherRouter, true);
dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
@@ -1044,11 +1070,11 @@ public class DAGAppMaster extends AbstractService {
protected TaskAttemptListener createTaskAttemptListener(AppContext context,
TaskHeartbeatHandler thh,
ContainerHeartbeatHandler chh,
- String[] taskCommunicatorClasses,
+ List<NamedEntityDescriptor> entityDescriptors,
boolean isLocal) {
TaskAttemptListener lis =
new TaskAttemptListenerImpTezDag(context, thh, chh,
- taskCommunicatorClasses, amConf, isLocal);
+ entityDescriptors, amConf, isLocal);
return lis;
}
@@ -1070,11 +1096,11 @@ public class DAGAppMaster extends AbstractService {
}
protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf,
- String[] containerLauncherClasses,
+ List<NamedEntityDescriptor> containerLauncherDescriptors,
boolean isLocal) throws
UnknownHostException {
return new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory,
- containerLauncherClasses, isLocal);
+ containerLauncherDescriptors, isLocal);
}
public ApplicationId getAppID() {
@@ -2134,7 +2160,16 @@ public class DAGAppMaster extends AbstractService {
// TODO Does this really need to be a YarnConfiguration ?
Configuration conf = new Configuration(new YarnConfiguration());
- TezUtilsInternal.addUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()), conf);
+
+ ConfigurationProto confProto =
+ TezUtilsInternal.readUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()));
+ TezUtilsInternal.addUserSpecifiedTezConfiguration(conf, confProto.getConfKeyValuesList());
+
+ AMPluginDescriptorProto amPluginDescriptorProto = null;
+ if (confProto.hasAmPluginDescriptor()) {
+ amPluginDescriptorProto = confProto.getAmPluginDescriptor();
+ }
+
UserGroupInformation.setConfiguration(conf);
Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
@@ -2146,7 +2181,7 @@ public class DAGAppMaster extends AbstractService {
System.getenv(Environment.PWD.name()),
TezCommonUtils.getTrimmedStrings(System.getenv(Environment.LOCAL_DIRS.name())),
TezCommonUtils.getTrimmedStrings(System.getenv(Environment.LOG_DIRS.name())),
- clientVersion, maxAppAttempts, credentials, jobUserName);
+ clientVersion, maxAppAttempts, credentials, jobUserName, amPluginDescriptorProto);
ShutdownHookManager.get().addShutdownHook(
new DAGAppMasterShutdownHook(appMaster), SHUTDOWN_HOOK_PRIORITY);
@@ -2252,7 +2287,7 @@ public class DAGAppMaster extends AbstractService {
LOG.info("Running DAG: " + dagPlan.getName());
String timeStamp = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(Calendar.getInstance().getTime());
- System.err.println(timeStamp + " Running Dag: "+ newDAG.getID());
+ System.err.println(timeStamp + " Running Dag: " + newDAG.getID());
System.out.println(timeStamp + " Running Dag: "+ newDAG.getID());
// Job name is the same as the app name until we support multiple dags
// for an app later
@@ -2358,60 +2393,51 @@ public class DAGAppMaster extends AbstractService {
TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT);
}
- // Tez default classnames are populated as TezConfiguration.TEZ_AM_SERVICE_PLUGINS_DEFAULT
- private String[] parsePlugins(BiMap<String, Integer> pluginMap, String[] pluginStrings,
- String context) {
- // TODO TEZ-2003 Duplicate error checking - ideally in the client itself. Depends on the final API.
- Preconditions.checkState(pluginStrings != null && pluginStrings.length > 0,
- "Plugin strings should not be null or empty: " + context);
-
- String[] classNames = new String[pluginStrings.length];
+ private static List<NamedEntityDescriptor> parsePlugin(
+ BiMap<String, Integer> pluginMap, List<TezNamedEntityDescriptorProto> namedEntityDescriptorProtos,
+ boolean tezYarnEnabled, boolean uberEnabled) {
int index = 0;
- for (String pluginString : pluginStrings) {
-
- String className;
- String identifierString;
-
- Preconditions.checkState(pluginString != null && !pluginString.isEmpty(),
- "Plugin string: " + pluginString + " should not be null or empty");
- if (pluginString.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) ||
- pluginString.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
- // Kind of ugly, but Tez internal routing is encoded via a String instead of classnames.
- // Individual components - TaskComm, Scheduler, Launcher deal with actual classname translation,
- // and avoid reflection.
- identifierString = pluginString;
- className = pluginString;
- } else {
- String[] parts = pluginString.split(":");
- Preconditions.checkState(
- parts.length == 2 && parts[0] != null && !parts[0].isEmpty() && parts[1] != null &&
- !parts[1].isEmpty(),
- "Invalid configuration string for " + context + ": " + pluginString);
- Preconditions.checkState(
- !parts[0].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) &&
- !parts[0].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT),
- "Identifier cannot be " + TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT + " or " +
- TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT + " for " +
- pluginString);
- identifierString = parts[0];
- className = parts[1];
- }
- pluginMap.put(identifierString, index);
- classNames[index] = className;
+
+ List<NamedEntityDescriptor> resultList = new LinkedList<>();
+
+ if (tezYarnEnabled) {
+ // Default classnames will be populated by individual components
+ NamedEntityDescriptor r = new NamedEntityDescriptor(
+ TezConstants.getTezYarnServicePluginName(), null);
+ resultList.add(r);
+ pluginMap.put(TezConstants.getTezYarnServicePluginName(), index);
+ index++;
+ }
+
+ if (uberEnabled) {
+ // Default classnames will be populated by individual components
+ NamedEntityDescriptor r = new NamedEntityDescriptor(
+ TezConstants.getTezUberServicePluginName(), null);
+ resultList.add(r);
+ pluginMap.put(TezConstants.getTezUberServicePluginName(), index);
index++;
}
- return classNames;
+
+ if (namedEntityDescriptorProtos != null) {
+ for (TezNamedEntityDescriptorProto namedEntityDescriptorProto : namedEntityDescriptorProtos) {
+ resultList.add(DagTypeConverters
+ .convertNamedDescriptorFromProto(namedEntityDescriptorProto));
+ pluginMap.put(resultList.get(index).getEntityName(), index);
+ index++;
+ }
+ }
+ return resultList;
}
- String buildPluginComponentLog(String[] classIdentifiers, BiMap<String, Integer> map,
+ String buildPluginComponentLog(List<NamedEntityDescriptor> namedEntityDescriptors, BiMap<String, Integer> map,
String component) {
StringBuilder sb = new StringBuilder();
sb.append("AM Level configured ").append(component).append(": ");
- for (int i = 0; i < classIdentifiers.length; i++) {
+ for (int i = 0; i < namedEntityDescriptors.size(); i++) {
sb.append("[").append(i).append(":").append(map.inverse().get(i))
- .append(":").append(classIdentifiers[i]).append("]");
- if (i != classIdentifiers.length - 1) {
+ .append(":").append(namedEntityDescriptors.get(i).getClassName()).append("]");
+ if (i != namedEntityDescriptors.size() - 1) {
sb.append(",");
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 599c208..1e34184 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -27,8 +27,11 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
import org.apache.commons.collections4.ListUtils;
import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
@@ -46,7 +49,6 @@ import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
-import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -100,28 +102,28 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
public TaskAttemptListenerImpTezDag(AppContext context,
TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
- String [] taskCommunicatorClassIdentifiers,
+ List<NamedEntityDescriptor> taskCommunicatorDescriptors,
Configuration conf,
boolean isPureLocalMode) {
super(TaskAttemptListenerImpTezDag.class.getName());
this.context = context;
this.taskHeartbeatHandler = thh;
this.containerHeartbeatHandler = chh;
- if (taskCommunicatorClassIdentifiers == null || taskCommunicatorClassIdentifiers.length == 0) {
+ if (taskCommunicatorDescriptors == null || taskCommunicatorDescriptors.isEmpty()) {
if (isPureLocalMode) {
- taskCommunicatorClassIdentifiers =
- new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+ taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
+ TezConstants.getTezUberServicePluginName(), null));
} else {
- taskCommunicatorClassIdentifiers =
- new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
+ TezConstants.getTezYarnServicePluginName(), null));
}
}
- this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
- this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorClassIdentifiers.length];
- this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorClassIdentifiers.length];
- for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
+ this.taskCommunicators = new TaskCommunicator[taskCommunicatorDescriptors.size()];
+ this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorDescriptors.size()];
+ this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorDescriptors.size()];
+ for (int i = 0 ; i < taskCommunicatorDescriptors.size() ; i++) {
taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, conf, i);
- taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
+ taskCommunicators[i] = createTaskCommunicator(taskCommunicatorDescriptors.get(i), i);
taskCommunicatorServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskCommunicators[i]);
}
// TODO TEZ-2118 Start using taskCommunicator indices properly
@@ -143,17 +145,18 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
}
- private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
- if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+ private TaskCommunicator createTaskCommunicator(NamedEntityDescriptor taskCommDescriptor, int taskCommIndex) {
+ if (taskCommDescriptor.getEntityName().equals(TezConstants.getTezYarnServicePluginName())) {
LOG.info("Using Default Task Communicator");
return createTezTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
- } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+ } else if (taskCommDescriptor.getEntityName().equals(TezConstants.getTezUberServicePluginName())) {
LOG.info("Using Default Local Task Communicator");
return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
} else {
- LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
+ // TODO TEZ-2003. Use the payload
+ LOG.info("Using TaskCommunicator {}:{} " + taskCommDescriptor.getEntityName(), taskCommDescriptor.getClassName());
Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
- .getClazz(taskCommClassIdentifier);
+ .getClazz(taskCommDescriptor.getClassName());
try {
Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
ctor.setAccessible(true);
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
index 458362f..335239e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
@@ -98,4 +98,6 @@ public interface DAG {
StateChangeNotifier getStateChangeNotifier();
+ org.apache.tez.dag.api.Vertex.VertexExecutionContext getDefaultExecutionContext();
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index ec2ef66..25518b0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -717,6 +717,15 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
}
@Override
+ public org.apache.tez.dag.api.Vertex.VertexExecutionContext getDefaultExecutionContext() {
+ if (jobPlan.hasDefaultExecutionContext()) {
+ return DagTypeConverters.convertFromProto(jobPlan.getDefaultExecutionContext());
+ } else {
+ return null;
+ }
+ }
+
+ @Override
public TezCounters getAllCounters() {
readLock.lock();
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index bdab984..2e8f218 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -995,14 +995,37 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
String tezDefaultComponentName =
- isLocal ? TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT :
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
- String taskSchedulerName =
- vertexConf.get(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, tezDefaultComponentName);
- String taskCommName = vertexConf
- .get(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, tezDefaultComponentName);
- String containerLauncherName = vertexConf
- .get(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, tezDefaultComponentName);
+ isLocal ? TezConstants.getTezUberServicePluginName() :
+ TezConstants.getTezYarnServicePluginName();
+
+ org.apache.tez.dag.api.Vertex.VertexExecutionContext execContext = dag.getDefaultExecutionContext();
+ if (vertexPlan.hasExecutionContext()) {
+ execContext = DagTypeConverters.convertFromProto(vertexPlan.getExecutionContext());
+ LOG.info("Using ExecutionContext from Vertex for Vertex {}", vertexName);
+ } else if (execContext != null) {
+ LOG.info("Using ExecutionContext from DAG for Vertex {}", vertexName);
+ }
+ if (execContext != null) {
+ if (execContext.shouldExecuteInAm()) {
+ tezDefaultComponentName = TezConstants.getTezUberServicePluginName();
+ }
+ }
+
+ String taskSchedulerName = tezDefaultComponentName;
+ String containerLauncherName = tezDefaultComponentName;
+ String taskCommName = tezDefaultComponentName;
+
+ if (execContext != null) {
+ if (execContext.getTaskSchedulerName() != null) {
+ taskSchedulerName = execContext.getTaskSchedulerName();
+ }
+ if (execContext.getContainerLauncherName() != null) {
+ containerLauncherName = execContext.getContainerLauncherName();
+ }
+ if (execContext.getTaskCommName() != null) {
+ taskCommName = execContext.getTaskCommName();
+ }
+ }
LOG.info("Vertex: " + logIdentifier + " configured with TaskScheduler=" + taskSchedulerName +
", ContainerLauncher=" + containerLauncherName + ", TaskComm=" + taskCommName);
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
index 34c7bc0..cba5c80 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
@@ -30,11 +30,11 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
import org.apache.tez.serviceplugins.api.ContainerStopRequest;
-import org.apache.tez.dag.api.TezConstants;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -269,8 +269,8 @@ public class ContainerLauncherImpl extends ContainerLauncher {
// nodes where containers will run at *this* point of time. This is
// *not* the cluster size and doesn't need to be.
- int numNodes = getContext().getNumNodes(
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+ int numNodes =
+ getContext().getNumNodes(TezConstants.getTezYarnServicePluginName());
int idealPoolSize = Math.min(limitOnPoolSize, numNodes);
if (poolSize < idealPoolSize) {
[23/51] [abbrv] tez git commit: TEZ-2381. Fixes after rebase 04/28.
(sseth)
Posted by ss...@apache.org.
TEZ-2381. Fixes after rebase 04/28. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/57745276
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/57745276
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/57745276
Branch: refs/heads/TEZ-2003
Commit: 57745276e1f8352c9a76693f737096bf6cff7b4c
Parents: 9d38581
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 28 13:41:12 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:34 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../dag/app/TaskAttemptListenerImpTezDag.java | 17 ++++----
.../app/TestTaskAttemptListenerImplTezDag.java | 44 +++++++++++++++-----
3 files changed, 42 insertions(+), 20 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/57745276/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index f6bc8e7..d42aaf8 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -18,5 +18,6 @@ ALL CHANGES:
TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
TEZ-2347. Expose additional information in TaskCommunicatorContext.
TEZ-2361. Propagate dag completion to TaskCommunicator.
+ TEZ-2381. Fixes after rebase 04/28.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/57745276/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 7cdf292..cbaed99 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -17,27 +17,21 @@
package org.apache.tez.dag.app;
-import javax.annotation.Nullable;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
-import java.net.InetSocketAddress;
-import java.net.URISyntaxException;
-import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
-import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.collections4.ListUtils;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
import org.apache.tez.runtime.api.impl.EventType;
-import com.google.common.base.Preconditions;
-import org.apache.tez.dag.api.event.VertexState;
import org.apache.tez.dag.api.event.VertexStateUpdate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -61,7 +55,6 @@ import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
import org.apache.tez.dag.app.rm.container.AMContainerTask;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -150,7 +143,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
LOG.info("Using Default Task Communicator");
- return new TezTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
+ return createTezTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
} else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Using Default Local Task Communicator");
return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
@@ -173,6 +166,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
}
}
+
+ @VisibleForTesting
+ protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
+ return new TezTaskCommunicatorImpl(context);
+ }
+
public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)
throws IOException, TezException {
ContainerId containerId = ConverterUtils.toContainerId(request
http://git-wip-us.apache.org/repos/asf/tez/blob/57745276/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index ab9fafe..2208220 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -47,11 +47,9 @@ import org.apache.tez.common.ContainerContext;
import org.apache.tez.common.ContainerTask;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
import org.apache.tez.dag.api.TezException;
import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
import org.apache.tez.common.TezTaskUmbilicalProtocol;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.app.dag.DAG;
@@ -82,7 +80,9 @@ import org.mockito.ArgumentCaptor;
// TODO TEZ-2003 Rename to TestTezTaskCommunicator
public class TestTaskAttemptListenerImplTezDag {
private ApplicationId appId;
+ private ApplicationAttemptId appAttemptId;
private AppContext appContext;
+ Credentials credentials;
AMContainerMap amContainerMap;
EventHandler eventHandler;
DAG dag;
@@ -98,11 +98,13 @@ public class TestTaskAttemptListenerImplTezDag {
@Before
public void setUp() {
appId = ApplicationId.newInstance(1000, 1);
+ appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
dag = mock(DAG.class);
TezDAGID dagID = TezDAGID.getInstance(appId, 1);
vertexID = TezVertexID.getInstance(dagID, 1);
taskID = TezTaskID.getInstance(vertexID, 1);
taskAttemptID = TezTaskAttemptID.getInstance(taskID, 1);
+ credentials = new Credentials();
amContainerMap = mock(AMContainerMap.class);
Map<ApplicationAccessType, String> appAcls = new HashMap<ApplicationAccessType, String>();
@@ -118,6 +120,8 @@ public class TestTaskAttemptListenerImplTezDag {
doReturn(amContainerMap).when(appContext).getAllContainers();
doReturn(clock).when(appContext).getClock();
+ doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
+ doReturn(credentials).when(appContext).getAppCredentials();
NodeId nodeId = NodeId.newInstance("localhost", 0);
AMContainer amContainer = mock(AMContainer.class);
Container container = mock(Container.class);
@@ -160,7 +164,7 @@ public class TestTaskAttemptListenerImplTezDag {
assertEquals(taskSpec, containerTask.getTaskSpec());
// Task unregistered. Should respond to heartbeats
- taskAttemptListener.unregisterTaskAttempt(taskAttemptId, 0);
+ taskAttemptListener.unregisterTaskAttempt(taskAttemptID, 0);
containerTask = tezUmbilical.getTask(containerContext2);
assertNull(containerTask);
@@ -190,7 +194,7 @@ public class TestTaskAttemptListenerImplTezDag {
TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
ContainerId containerId1 = createContainerId(appId, 1);
- doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
+
ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
taskAttemptListener.registerRunningContainer(containerId1, 0);
containerTask = tezUmbilical.getTask(containerContext1);
@@ -320,7 +324,6 @@ public class TestTaskAttemptListenerImplTezDag {
int fromEventId, int maxEvents, int nextFromEventId,
List<TezEvent> sendEvents) throws IOException, TezException {
ContainerId containerId = createContainerId(appId, 1);
- long requestId = 0;
Vertex vertex = mock(Vertex.class);
doReturn(vertex).when(dag).getVertex(vertexID);
@@ -328,13 +331,13 @@ public class TestTaskAttemptListenerImplTezDag {
TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(nextFromEventId, sendEvents, 0);
doReturn(eventInfo).when(vertex).getTaskAttemptTezEvents(taskAttemptID, fromEventId, 0, maxEvents);
- taskAttemptListener.registerRunningContainer(containerId);
- taskAttemptListener.registerTaskAttempt(amContainerTask, containerId);
+ taskAttemptListener.registerRunningContainer(containerId, 0);
+ taskAttemptListener.registerTaskAttempt(amContainerTask, containerId, 0);
+
+ TaskHeartbeatRequest request = mock(TaskHeartbeatRequest.class);
- TezHeartbeatRequest request = mock(TezHeartbeatRequest.class);
doReturn(containerId.toString()).when(request).getContainerIdentifier();
- doReturn(taskAttemptID).when(request).getCurrentTaskAttemptID();
- doReturn(++requestId).when(request).getRequestId();
+ doReturn(taskAttemptID).when(request).getTaskAttemptId();
doReturn(events).when(request).getEvents();
doReturn(maxEvents).when(request).getMaxEvents();
doReturn(fromEventId).when(request).getStartIndex();
@@ -348,6 +351,25 @@ public class TestTaskAttemptListenerImplTezDag {
return ContainerId.newInstance(appAttemptId, containerIdx);
}
+ private static class TaskAttemptListenerImplForTest extends TaskAttemptListenerImpTezDag {
+
+ public TaskAttemptListenerImplForTest(AppContext context,
+ TaskHeartbeatHandler thh,
+ ContainerHeartbeatHandler chh,
+ JobTokenSecretManager jobTokenSecretManager,
+ String[] taskCommunicatorClassIdentifiers,
+ boolean isPureLocalMode) {
+ super(context, thh, chh, jobTokenSecretManager, taskCommunicatorClassIdentifiers,
+ isPureLocalMode);
+ }
+
+ @Override
+ protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
+ return new TezTaskCommunicatorImplForTest(context);
+ }
+
+ }
+
private static class TezTaskCommunicatorImplForTest extends TezTaskCommunicatorImpl {
public TezTaskCommunicatorImplForTest(
[35/51] [abbrv] tez git commit: TEZ-2124. Change Node tracking to
work per external container source. (sseth)
Posted by ss...@apache.org.
TEZ-2124. Change Node tracking to work per external container source. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/8a03f113
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/8a03f113
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/8a03f113
Branch: refs/heads/TEZ-2003
Commit: 8a03f113079ce581e52f702a3324979ad0710abc
Parents: 0539d31
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Jul 16 14:18:35 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../apache/tez/dag/app/dag/impl/DAGImpl.java | 2 +
.../apache/tez/dag/app/dag/impl/TaskImpl.java | 3 +-
.../dag/app/launcher/ContainerLauncherImpl.java | 5 +-
.../app/rm/TaskSchedulerAppCallbackImpl.java | 89 +++++++++
.../dag/app/rm/TaskSchedulerEventHandler.java | 71 +++----
.../apache/tez/dag/app/rm/node/AMNodeEvent.java | 8 +-
.../rm/node/AMNodeEventContainerAllocated.java | 4 +-
.../rm/node/AMNodeEventNodeCountUpdated.java | 4 +-
.../app/rm/node/AMNodeEventStateChanged.java | 4 +-
.../rm/node/AMNodeEventTaskAttemptEnded.java | 4 +-
.../node/AMNodeEventTaskAttemptSucceeded.java | 4 +-
.../apache/tez/dag/app/rm/node/AMNodeImpl.java | 6 +-
.../tez/dag/app/rm/node/AMNodeTracker.java | 162 +++++-----------
.../dag/app/rm/node/PerSourceNodeTracker.java | 187 +++++++++++++++++++
.../apache/tez/dag/app/MockDAGAppMaster.java | 2 +-
.../tez/dag/app/TestMockDAGAppMaster.java | 2 +-
.../tez/dag/app/rm/TestContainerReuse.java | 62 +++---
.../app/rm/TestTaskSchedulerEventHandler.java | 11 +-
.../dag/app/rm/TestTaskSchedulerHelpers.java | 2 +-
.../tez/dag/app/rm/node/TestAMNodeTracker.java | 64 ++++---
21 files changed, 462 insertions(+), 235 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 590fe7f..604947c 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -33,5 +33,6 @@ ALL CHANGES:
TEZ-2508. rebase 06/01
TEZ-2526. Fix version for tez-history-parser.
TEZ-2621. rebase 07/14
+ TEZ-2124. Change Node tracking to work per external container source.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index e37fc2f..ec2ef66 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -1440,9 +1440,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
VertexImpl v = createVertex(this, vertexName, i);
addVertex(v);
}
+
// check task resources, only check it in non-local mode
if (!appContext.isLocal()) {
for (Vertex v : vertexMap.values()) {
+ // TODO TEZ-2003 (post) Ideally, this should be per source.
if (v.getTaskResource().compareTo(appContext.getClusterInfo().getMaxContainerCapability()) > 0) {
String msg = "Vertex's TaskResource is beyond the cluster container capability," +
"Vertex=" + v.getLogIdentifier() +", Requested TaskResource=" + v.getTaskResource()
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
index 93b4c3f..1b55295 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
@@ -1396,7 +1396,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
if (amContainer != null) {
// inform the node about failure
task.eventHandler.handle(
- new AMNodeEventTaskAttemptEnded(amContainer.getContainer().getNodeId(),
+ new AMNodeEventTaskAttemptEnded(amContainer.getContainer().getNodeId(),
+ task.getVertex().getTaskSchedulerIdentifier(),
containerId, failedAttemptId, true));
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
index a1eb2a7..a12fb04 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
@@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tez.dag.api.TezConstants;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -292,7 +293,9 @@ public class ContainerLauncherImpl extends AbstractService implements
// nodes where containers will run at *this* point of time. This is
// *not* the cluster size and doesn't need to be.
- int numNodes = context.getNodeTracker().getNumNodes();
+ int yarnSourceIndex =
+ context.getTaskScheduerIdentifier(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+ int numNodes = context.getNodeTracker().getNumNodes(yarnSourceIndex);
int idealPoolSize = Math.min(limitOnPoolSize, numNodes);
if (poolSize < idealPoolSize) {
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
new file mode 100644
index 0000000..ea37e94
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
@@ -0,0 +1,89 @@
+/*
+ * 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.tez.dag.app.rm;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class TaskSchedulerAppCallbackImpl implements TaskSchedulerService.TaskSchedulerAppCallback{
+
+ private final TaskSchedulerEventHandler tseh;
+ private final int schedulerId;
+
+ public TaskSchedulerAppCallbackImpl(TaskSchedulerEventHandler tseh, int schedulerId) {
+ this.tseh = tseh;
+ this.schedulerId = schedulerId;
+ }
+
+ @Override
+ public void taskAllocated(Object task, Object appCookie, Container container) {
+ tseh.taskAllocated(schedulerId, task, appCookie, container);
+ }
+
+ @Override
+ public void containerCompleted(Object taskLastAllocated, ContainerStatus containerStatus) {
+ tseh.containerCompleted(schedulerId, taskLastAllocated, containerStatus);
+ }
+
+ @Override
+ public void containerBeingReleased(ContainerId containerId) {
+ tseh.containerBeingReleased(schedulerId, containerId);
+ }
+
+ @Override
+ public void nodesUpdated(List<NodeReport> updatedNodes) {
+ tseh.nodesUpdated(schedulerId, updatedNodes);
+ }
+
+ @Override
+ public void appShutdownRequested() {
+ tseh.appShutdownRequested(schedulerId);
+ }
+
+ @Override
+ public void setApplicationRegistrationData(Resource maxContainerCapability,
+ Map<ApplicationAccessType, String> appAcls,
+ ByteBuffer clientAMSecretKey) {
+ tseh.setApplicationRegistrationData(schedulerId, maxContainerCapability, appAcls, clientAMSecretKey);
+ }
+
+ @Override
+ public void onError(Throwable t) {
+ tseh.onError(schedulerId, t);
+ }
+
+ @Override
+ public float getProgress() {
+ return tseh.getProgress(schedulerId);
+ }
+
+ @Override
+ public void preemptContainer(ContainerId containerId) {
+ tseh.preemptContainer(schedulerId, containerId);
+ }
+
+ @Override
+ public AppFinalStatus getFinalAppStatus() {
+ return tseh.getFinalAppStatus();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 69763b3..1ad0059 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -29,6 +29,7 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.annotations.VisibleForTesting;
+import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -81,8 +82,7 @@ import org.apache.tez.dag.records.TaskAttemptTerminationCause;
import com.google.common.base.Preconditions;
-public class TaskSchedulerEventHandler extends AbstractService
- implements TaskSchedulerAppCallback,
+public class TaskSchedulerEventHandler extends AbstractService implements
EventHandler<AMSchedulerEvent> {
static final Logger LOG = LoggerFactory.getLogger(TaskSchedulerEventHandler.class);
@@ -315,7 +315,7 @@ public class TaskSchedulerEventHandler extends AbstractService
// stopped.
// AMNodeImpl blacklisting logic does not account for KILLED attempts.
sendEvent(new AMNodeEventTaskAttemptEnded(appContext.getAllContainers().
- get(attemptContainerId).getContainer().getNodeId(), attemptContainerId,
+ get(attemptContainerId).getContainer().getNodeId(), event.getSchedulerId(), attemptContainerId,
attempt.getID(), event.getState() == TaskAttemptState.FAILED));
}
}
@@ -330,7 +330,7 @@ public class TaskSchedulerEventHandler extends AbstractService
sendEvent(new AMContainerEventTASucceeded(usedContainerId,
event.getAttemptID()));
sendEvent(new AMNodeEventTaskAttemptSucceeded(appContext.getAllContainers().
- get(usedContainerId).getContainer().getNodeId(), usedContainerId,
+ get(usedContainerId).getContainer().getNodeId(), event.getSchedulerId(), usedContainerId,
event.getAttemptID()));
}
@@ -392,14 +392,16 @@ public class TaskSchedulerEventHandler extends AbstractService
private TaskSchedulerService createTaskScheduler(String host, int port, String trackingUrl,
AppContext appContext,
String schedulerClassName,
- long customAppIdIdentifier) {
+ long customAppIdIdentifier,
+ int schedulerId) {
+ TaskSchedulerAppCallback appCallback = new TaskSchedulerAppCallbackImpl(this, schedulerId);
if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
- return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
+ return new YarnTaskSchedulerService(appCallback, this.containerSignatureMatcher,
host, port, trackingUrl, appContext);
} else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Creating TaskScheduler: Local TaskScheduler");
- return new LocalTaskSchedulerService(this, this.containerSignatureMatcher,
+ return new LocalTaskSchedulerService(appCallback, this.containerSignatureMatcher,
host, port, trackingUrl, customAppIdIdentifier, appContext);
} else {
LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
@@ -411,7 +413,7 @@ public class TaskSchedulerEventHandler extends AbstractService
.getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
int.class, String.class, long.class, Configuration.class);
ctor.setAccessible(true);
- return ctor.newInstance(this, appContext, host, port, trackingUrl, customAppIdIdentifier,
+ return ctor.newInstance(appCallback, appContext, host, port, trackingUrl, customAppIdIdentifier,
getConfig());
} catch (NoSuchMethodException e) {
throw new TezUncheckedException(e);
@@ -441,7 +443,7 @@ public class TaskSchedulerEventHandler extends AbstractService
LOG.info("ClusterIdentifier for TaskScheduler [" + i + ":" + taskSchedulerClasses[i] + "]=" +
customAppIdIdentifier);
taskSchedulers[i] = createTaskScheduler(host, port,
- trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier);
+ trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier, i);
}
}
@@ -521,20 +523,21 @@ public class TaskSchedulerEventHandler extends AbstractService
}
}
- // TaskSchedulerAppCallback methods
- @Override
- public synchronized void taskAllocated(Object task,
+ // TODO TEZ-2003 Consolidate TaskSchedulerAppCallback methods once these methods are moved into context
+
+ // TaskSchedulerAppCallback methods with schedulerId, where relevant
+ public synchronized void taskAllocated(int schedulerId, Object task,
Object appCookie,
Container container) {
AMSchedulerEventTALaunchRequest event =
(AMSchedulerEventTALaunchRequest) appCookie;
ContainerId containerId = container.getId();
if (appContext.getAllContainers()
- .addContainerIfNew(container, event.getSchedulerId(), event.getLauncherId(),
+ .addContainerIfNew(container, schedulerId, event.getLauncherId(),
event.getTaskCommId())) {
- appContext.getNodeTracker().nodeSeen(container.getNodeId());
+ appContext.getNodeTracker().nodeSeen(container.getNodeId(), schedulerId);
sendEvent(new AMNodeEventContainerAllocated(container
- .getNodeId(), container.getId()));
+ .getNodeId(), schedulerId, container.getId()));
}
@@ -554,8 +557,8 @@ public class TaskSchedulerEventHandler extends AbstractService
.getContainerContext().getCredentials(), event.getPriority()));
}
- @Override
- public synchronized void containerCompleted(Object task, ContainerStatus containerStatus) {
+ public synchronized void containerCompleted(int schedulerId, Object task, ContainerStatus containerStatus) {
+ // SchedulerId isn't used here since no node updates are sent out
// Inform the Containers about completion.
AMContainer amContainer = appContext.getAllContainers().get(containerStatus.getContainerId());
if (amContainer != null) {
@@ -578,8 +581,8 @@ public class TaskSchedulerEventHandler extends AbstractService
}
}
- @Override
- public synchronized void containerBeingReleased(ContainerId containerId) {
+ public synchronized void containerBeingReleased(int schedulerId, ContainerId containerId) {
+ // SchedulerId isn't used here since no node updates are sent out
AMContainer amContainer = appContext.getAllContainers().get(containerId);
if (amContainer != null) {
sendEvent(new AMContainerEventStopRequest(containerId));
@@ -587,28 +590,27 @@ public class TaskSchedulerEventHandler extends AbstractService
}
@SuppressWarnings("unchecked")
- @Override
- public synchronized void nodesUpdated(List<NodeReport> updatedNodes) {
+ public synchronized void nodesUpdated(int schedulerId, List<NodeReport> updatedNodes) {
for (NodeReport nr : updatedNodes) {
// Scheduler will find out from the node, if at all.
// Relying on the RM to not allocate containers on an unhealthy node.
- eventHandler.handle(new AMNodeEventStateChanged(nr));
+ eventHandler.handle(new AMNodeEventStateChanged(nr, schedulerId));
}
}
- @Override
- public synchronized void appShutdownRequested() {
+ public synchronized void appShutdownRequested(int schedulerId) {
// This can happen if the RM has been restarted. If it is in that state,
// this application must clean itself up.
- LOG.info("App shutdown requested by scheduler");
+ LOG.info("App shutdown requested by scheduler {}", schedulerId);
sendEvent(new DAGAppMasterEvent(DAGAppMasterEventType.AM_REBOOT));
}
- @Override
public synchronized void setApplicationRegistrationData(
+ int schedulerId,
Resource maxContainerCapability,
Map<ApplicationAccessType, String> appAcls,
ByteBuffer clientAMSecretKey) {
+ // TODO TEZ-2003 (post) Ideally clusterInfo should be available per source rather than a global view.
this.appContext.getClusterInfo().setMaxContainerCapability(
maxContainerCapability);
this.appAcls = appAcls;
@@ -619,7 +621,6 @@ public class TaskSchedulerEventHandler extends AbstractService
// TaskScheduler uses a separate thread for it's callbacks. Since this method
// returns a value which is required, the TaskScheduler wait for the call to
// complete and can hence lead to a deadlock if called from within a TSEH lock.
- @Override
public AppFinalStatus getFinalAppStatus() {
FinalApplicationStatus finishState = FinalApplicationStatus.UNDEFINED;
StringBuffer sb = new StringBuffer();
@@ -661,24 +662,25 @@ public class TaskSchedulerEventHandler extends AbstractService
// TaskScheduler uses a separate thread for it's callbacks. Since this method
// returns a value which is required, the TaskScheduler wait for the call to
// complete and can hence lead to a deadlock if called from within a TSEH lock.
- @Override
- public float getProgress() {
+ public float getProgress(int schedulerId) {
// at this point allocate has been called and so node count must be available
// may change after YARN-1722
// This is a heartbeat in from the scheduler into the APP, and is being used to piggy-back and
// node updates from the cluster.
+
+ // Doubles as a mechanism to update node counts periodically. Hence schedulerId required.
+
// TODO Handle this in TEZ-2124. Need a way to know which scheduler is calling in.
int nodeCount = taskSchedulers[0].getClusterNodeCount();
if (nodeCount != cachedNodeCount) {
cachedNodeCount = nodeCount;
- sendEvent(new AMNodeEventNodeCountUpdated(cachedNodeCount));
+ sendEvent(new AMNodeEventNodeCountUpdated(cachedNodeCount, schedulerId));
}
return dagAppMaster.getProgress();
}
- @Override
- public void onError(Throwable t) {
- LOG.info("Error reported by scheduler", t);
+ public void onError(int schedulerId, Throwable t) {
+ LOG.info("Error reported by scheduler {} - {}", schedulerId, t);
sendEvent(new DAGAppMasterEventSchedulingServiceError(t));
}
@@ -693,8 +695,7 @@ public class TaskSchedulerEventHandler extends AbstractService
// the context has updated information.
}
- @Override
- public void preemptContainer(ContainerId containerId) {
+ public void preemptContainer(int schedulerId, ContainerId containerId) {
// TODO Why is this making a call back into the scheduler, when the call is originating from there.
// An AMContainer instance should already exist if an attempt is being made to preempt it
AMContainer amContainer = appContext.getAllContainers().get(containerId);
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
index a623cda..85bc513 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEvent.java
@@ -24,13 +24,19 @@ import org.apache.hadoop.yarn.event.AbstractEvent;
public class AMNodeEvent extends AbstractEvent<AMNodeEventType> {
private final NodeId nodeId;
+ private final int sourceId; // Effectively the schedulerId
- public AMNodeEvent(NodeId nodeId, AMNodeEventType type) {
+ public AMNodeEvent(NodeId nodeId, int sourceId, AMNodeEventType type) {
super(type);
this.nodeId = nodeId;
+ this.sourceId = sourceId;
}
public NodeId getNodeId() {
return this.nodeId;
}
+
+ public int getSourceId() {
+ return sourceId;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java
index 0770969..e250f42 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerAllocated.java
@@ -24,8 +24,8 @@ public class AMNodeEventContainerAllocated extends AMNodeEvent {
private final ContainerId containerId;
- public AMNodeEventContainerAllocated(NodeId nodeId, ContainerId containerId) {
- super(nodeId, AMNodeEventType.N_CONTAINER_ALLOCATED);
+ public AMNodeEventContainerAllocated(NodeId nodeId, int sourceId, ContainerId containerId) {
+ super(nodeId, sourceId, AMNodeEventType.N_CONTAINER_ALLOCATED);
this.containerId = containerId;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java
index 86ca1fc..3b35daf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventNodeCountUpdated.java
@@ -22,8 +22,8 @@ public class AMNodeEventNodeCountUpdated extends AMNodeEvent {
private final int count;
- public AMNodeEventNodeCountUpdated(int nodeCount) {
- super(null, AMNodeEventType.N_NODE_COUNT_UPDATED);
+ public AMNodeEventNodeCountUpdated(int nodeCount, int sourceId) {
+ super(null, sourceId, AMNodeEventType.N_NODE_COUNT_UPDATED);
this.count = nodeCount;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java
index ca4e5bd..b371ddd 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventStateChanged.java
@@ -23,8 +23,8 @@ public class AMNodeEventStateChanged extends AMNodeEvent {
private NodeReport nodeReport;
- public AMNodeEventStateChanged(NodeReport nodeReport) {
- super(nodeReport.getNodeId(),
+ public AMNodeEventStateChanged(NodeReport nodeReport, int sourceId) {
+ super(nodeReport.getNodeId(), sourceId,
(nodeReport.getNodeState().isUnusable() ?
AMNodeEventType.N_TURNED_UNHEALTHY :
AMNodeEventType.N_TURNED_HEALTHY));
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java
index c823236..4a4cb61 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptEnded.java
@@ -27,9 +27,9 @@ public class AMNodeEventTaskAttemptEnded extends AMNodeEvent {
private final ContainerId containerId;
private final TezTaskAttemptID taskAttemptId;
- public AMNodeEventTaskAttemptEnded(NodeId nodeId, ContainerId containerId,
+ public AMNodeEventTaskAttemptEnded(NodeId nodeId, int sourceId, ContainerId containerId,
TezTaskAttemptID taskAttemptId, boolean failed) {
- super(nodeId, AMNodeEventType.N_TA_ENDED);
+ super(nodeId, sourceId, AMNodeEventType.N_TA_ENDED);
this.failed = failed;
this.containerId = containerId;
this.taskAttemptId = taskAttemptId;
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java
index b07d594..2b8cb7d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventTaskAttemptSucceeded.java
@@ -27,9 +27,9 @@ public class AMNodeEventTaskAttemptSucceeded extends AMNodeEvent {
private final ContainerId containerId;
private final TezTaskAttemptID taskAttemptId;
- public AMNodeEventTaskAttemptSucceeded(NodeId nodeId,
+ public AMNodeEventTaskAttemptSucceeded(NodeId nodeId, int sourceId,
ContainerId containerId, TezTaskAttemptID taskAttemptId) {
- super(nodeId, AMNodeEventType.N_TA_SUCCEEDED);
+ super(nodeId, sourceId, AMNodeEventType.N_TA_SUCCEEDED);
this.containerId = containerId;
this.taskAttemptId = taskAttemptId;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
index 0d8e4cd..88b36cb1f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
@@ -54,6 +54,7 @@ public class AMNodeImpl implements AMNode {
private final ReadLock readLock;
private final WriteLock writeLock;
private final NodeId nodeId;
+ private final int sourceId;
private final AppContext appContext;
private final int maxTaskFailuresPerNode;
private boolean blacklistingEnabled;
@@ -172,13 +173,14 @@ public class AMNodeImpl implements AMNode {
@SuppressWarnings("rawtypes")
- public AMNodeImpl(NodeId nodeId, int maxTaskFailuresPerNode,
+ public AMNodeImpl(NodeId nodeId, int sourceId, int maxTaskFailuresPerNode,
EventHandler eventHandler, boolean blacklistingEnabled,
AppContext appContext) {
ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
this.readLock = rwLock.readLock();
this.writeLock = rwLock.writeLock();
this.nodeId = nodeId;
+ this.sourceId = sourceId;
this.appContext = appContext;
this.eventHandler = eventHandler;
this.blacklistingEnabled = blacklistingEnabled;
@@ -247,7 +249,7 @@ public class AMNodeImpl implements AMNode {
/* Blacklist the node with the AMNodeTracker and check if the node should be blacklisted */
protected boolean registerBadNodeAndShouldBlacklist() {
- return appContext.getNodeTracker().registerBadNodeAndShouldBlacklist(this);
+ return appContext.getNodeTracker().registerBadNodeAndShouldBlacklist(this, sourceId);
}
protected void blacklistSelf() {
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
index 102cbe9..0668ff2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
@@ -18,9 +18,8 @@
package org.apache.tez.dag.app.rm.node;
-import java.util.HashSet;
-import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import org.apache.tez.dag.app.dag.DAG;
import org.slf4j.Logger;
@@ -29,7 +28,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
@@ -42,23 +40,21 @@ public class AMNodeTracker extends AbstractService implements
static final Logger LOG = LoggerFactory.getLogger(AMNodeTracker.class);
- private final ConcurrentHashMap<NodeId, AMNode> nodeMap;
- private final ConcurrentHashMap<String, Set<NodeId>> blacklistMap;
+ private final ConcurrentMap<Integer, PerSourceNodeTracker> perSourceNodeTrackers;
+
@SuppressWarnings("rawtypes")
private final EventHandler eventHandler;
private final AppContext appContext;
- private int numClusterNodes;
- private boolean ignoreBlacklisting = false;
+
+ // Not final since it's setup in serviceInit
private int maxTaskFailuresPerNode;
private boolean nodeBlacklistingEnabled;
private int blacklistDisablePercent;
- float currentIgnoreBlacklistingCountThreshold = 0;
-
+
@SuppressWarnings("rawtypes")
public AMNodeTracker(EventHandler eventHandler, AppContext appContext) {
super("AMNodeMap");
- this.nodeMap = new ConcurrentHashMap<NodeId, AMNode>();
- this.blacklistMap = new ConcurrentHashMap<String, Set<NodeId>>();
+ this.perSourceNodeTrackers = new ConcurrentHashMap<>();
this.eventHandler = eventHandler;
this.appContext = appContext;
}
@@ -76,7 +72,7 @@ public class AMNodeTracker extends AbstractService implements
TezConfiguration.TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD_DEFAULT);
LOG.info("blacklistDisablePercent is " + blacklistDisablePercent +
- ", blacklistingEnabled: " + nodeBlacklistingEnabled +
+ ", blacklistingEnabled: " + nodeBlacklistingEnabled +
", maxTaskFailuresPerNode: " + maxTaskFailuresPerNode);
if (blacklistDisablePercent < -1 || blacklistDisablePercent > 100) {
@@ -85,130 +81,66 @@ public class AMNodeTracker extends AbstractService implements
+ ". Should be an integer between 0 and 100 or -1 to disabled");
}
}
-
- public void nodeSeen(NodeId nodeId) {
- if (nodeMap.putIfAbsent(nodeId, new AMNodeImpl(nodeId, maxTaskFailuresPerNode,
- eventHandler, nodeBlacklistingEnabled, appContext)) == null) {
- LOG.info("Adding new node: " + nodeId);
- }
- }
- private void addToBlackList(NodeId nodeId) {
- String host = nodeId.getHost();
-
- if (!blacklistMap.containsKey(host)) {
- blacklistMap.putIfAbsent(host, new HashSet<NodeId>());
- }
- Set<NodeId> nodes = blacklistMap.get(host);
-
- if (!nodes.contains(nodeId)) {
- nodes.add(nodeId);
- }
+ public void nodeSeen(NodeId nodeId, int sourceId) {
+ PerSourceNodeTracker nodeTracker = getAndCreateIfNeededPerSourceTracker(sourceId);
+ nodeTracker.nodeSeen(nodeId);
}
- boolean registerBadNodeAndShouldBlacklist(AMNode amNode) {
- if (nodeBlacklistingEnabled) {
- addToBlackList(amNode.getNodeId());
- computeIgnoreBlacklisting();
- return !ignoreBlacklisting;
- } else {
- return false;
- }
+
+ boolean registerBadNodeAndShouldBlacklist(AMNode amNode, int sourceId) {
+ return perSourceNodeTrackers.get(sourceId).registerBadNodeAndShouldBlacklist(amNode);
}
public void handle(AMNodeEvent rEvent) {
// No synchronization required until there's multiple dispatchers.
- NodeId nodeId = rEvent.getNodeId();
switch (rEvent.getType()) {
- case N_NODE_COUNT_UPDATED:
- AMNodeEventNodeCountUpdated event = (AMNodeEventNodeCountUpdated) rEvent;
- numClusterNodes = event.getNodeCount();
- LOG.info("Num cluster nodes = " + numClusterNodes);
- recomputeCurrentIgnoreBlacklistingThreshold();
- computeIgnoreBlacklisting();
- break;
- case N_TURNED_UNHEALTHY:
- case N_TURNED_HEALTHY:
- AMNode amNode = nodeMap.get(nodeId);
- if (amNode == null) {
- LOG.info("Ignoring RM Health Update for unknown node: " + nodeId);
- } else {
- amNode.handle(rEvent);
- }
- break;
- default:
- nodeMap.get(nodeId).handle(rEvent);
+ case N_CONTAINER_ALLOCATED:
+ case N_TA_SUCCEEDED:
+ case N_TA_ENDED:
+ case N_IGNORE_BLACKLISTING_ENABLED:
+ case N_IGNORE_BLACKLISTING_DISABLED:
+ // All of these will only be seen after a node has been registered.
+ perSourceNodeTrackers.get(rEvent.getSourceId()).handle(rEvent);
+ break;
+ case N_TURNED_UNHEALTHY:
+ case N_TURNED_HEALTHY:
+ case N_NODE_COUNT_UPDATED:
+ // These events can be seen without a node having been marked as 'seen' before
+ getAndCreateIfNeededPerSourceTracker(rEvent.getSourceId()).handle(rEvent);
+ break;
}
}
- private void recomputeCurrentIgnoreBlacklistingThreshold() {
- if (nodeBlacklistingEnabled && blacklistDisablePercent != -1) {
- currentIgnoreBlacklistingCountThreshold =
- (float) numClusterNodes * blacklistDisablePercent / 100;
- }
+ public AMNode get(NodeId nodeId, int sourceId) {
+ return perSourceNodeTrackers.get(sourceId).get(nodeId);
}
- // May be incorrect if there's multiple NodeManagers running on a single host.
- // knownNodeCount is based on node managers, not hosts. blacklisting is
- // currently based on hosts.
- protected void computeIgnoreBlacklisting() {
-
- boolean stateChanged = false;
-
- if (!nodeBlacklistingEnabled || blacklistDisablePercent == -1 || blacklistMap.size() == 0) {
- return;
- }
- if (blacklistMap.size() >= currentIgnoreBlacklistingCountThreshold) {
- if (ignoreBlacklisting == false) {
- ignoreBlacklisting = true;
- LOG.info("Ignore Blacklisting set to true. Known: " + numClusterNodes
- + ", Blacklisted: " + blacklistMap.size());
- stateChanged = true;
- }
- } else {
- if (ignoreBlacklisting == true) {
- ignoreBlacklisting = false;
- LOG.info("Ignore blacklisting set to false. Known: "
- + numClusterNodes + ", Blacklisted: " + blacklistMap.size());
- stateChanged = true;
- }
- }
-
- if (stateChanged) {
- sendIngoreBlacklistingStateToNodes();
- }
- }
-
- private void sendIngoreBlacklistingStateToNodes() {
- AMNodeEventType eventType =
- ignoreBlacklisting ? AMNodeEventType.N_IGNORE_BLACKLISTING_ENABLED
- : AMNodeEventType.N_IGNORE_BLACKLISTING_DISABLED;
- for (NodeId nodeId : nodeMap.keySet()) {
- sendEvent(new AMNodeEvent(nodeId, eventType));
- }
- }
-
- public AMNode get(NodeId nodeId) {
- return nodeMap.get(nodeId);
- }
-
- @SuppressWarnings("unchecked")
- private void sendEvent(Event<?> event) {
- this.eventHandler.handle(event);
- }
-
- public int getNumNodes() {
- return nodeMap.size();
+ public int getNumNodes(int sourceId) {
+ return perSourceNodeTrackers.get(sourceId).getNumNodes();
}
@Private
@VisibleForTesting
- public boolean isBlacklistingIgnored() {
- return this.ignoreBlacklisting;
+ public boolean isBlacklistingIgnored(int sourceId) {
+ return perSourceNodeTrackers.get(sourceId).isBlacklistingIgnored();
}
public void dagComplete(DAG dag) {
// TODO TEZ-2337 Maybe reset failures from previous DAGs
}
+ private PerSourceNodeTracker getAndCreateIfNeededPerSourceTracker(int sourceId) {
+ PerSourceNodeTracker nodeTracker = perSourceNodeTrackers.get(sourceId);
+ if (nodeTracker == null) {
+ nodeTracker =
+ new PerSourceNodeTracker(sourceId, eventHandler, appContext, maxTaskFailuresPerNode,
+ nodeBlacklistingEnabled, blacklistDisablePercent);
+ PerSourceNodeTracker old = perSourceNodeTrackers.putIfAbsent(sourceId, nodeTracker);
+ nodeTracker = old != null ? old : nodeTracker;
+ }
+ return nodeTracker;
+ }
+
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
new file mode 100644
index 0000000..3264708
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
@@ -0,0 +1,187 @@
+/*
+ * 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.tez.dag.app.rm.node;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.dag.app.AppContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PerSourceNodeTracker {
+
+ static final Logger LOG = LoggerFactory.getLogger(PerSourceNodeTracker.class);
+
+ private final int sourceId;
+ private final ConcurrentHashMap<NodeId, AMNode> nodeMap;
+ private final ConcurrentHashMap<String, Set<NodeId>> blacklistMap;
+
+ @SuppressWarnings("rawtypes")
+ private final EventHandler eventHandler;
+ private final AppContext appContext;
+
+ private final int maxTaskFailuresPerNode;
+ private final boolean nodeBlacklistingEnabled;
+ private final int blacklistDisablePercent;
+
+ private int numClusterNodes;
+ float currentIgnoreBlacklistingCountThreshold = 0;
+ private boolean ignoreBlacklisting = false;
+
+ @SuppressWarnings("rawtypes")
+ public PerSourceNodeTracker(int sourceId, EventHandler eventHandler, AppContext appContext,
+ int maxTaskFailuresPerNode, boolean nodeBlacklistingEnabled,
+ int blacklistDisablePercent) {
+ this.sourceId = sourceId;
+ this.nodeMap = new ConcurrentHashMap<>();
+ this.blacklistMap = new ConcurrentHashMap<>();
+ this.eventHandler = eventHandler;
+ this.appContext = appContext;
+
+ this.maxTaskFailuresPerNode = maxTaskFailuresPerNode;
+ this.nodeBlacklistingEnabled = nodeBlacklistingEnabled;
+ this.blacklistDisablePercent = blacklistDisablePercent;
+ }
+
+
+
+ public void nodeSeen(NodeId nodeId) {
+ if (nodeMap.putIfAbsent(nodeId, new AMNodeImpl(nodeId, sourceId, maxTaskFailuresPerNode,
+ eventHandler, nodeBlacklistingEnabled, appContext)) == null) {
+ LOG.info("Adding new node {} to nodeTracker {}", nodeId, sourceId);
+ }
+ }
+
+ public AMNode get(NodeId nodeId) {
+ return nodeMap.get(nodeId);
+ }
+
+ public int getNumNodes() {
+ return nodeMap.size();
+ }
+
+ public void handle(AMNodeEvent rEvent) {
+ // No synchronization required until there's multiple dispatchers.
+ NodeId nodeId = rEvent.getNodeId();
+ switch (rEvent.getType()) {
+ case N_NODE_COUNT_UPDATED:
+ AMNodeEventNodeCountUpdated event = (AMNodeEventNodeCountUpdated) rEvent;
+ numClusterNodes = event.getNodeCount();
+ LOG.info("Num cluster nodes = " + numClusterNodes);
+ recomputeCurrentIgnoreBlacklistingThreshold();
+ computeIgnoreBlacklisting();
+ break;
+ case N_TURNED_UNHEALTHY:
+ case N_TURNED_HEALTHY:
+ AMNode amNode = nodeMap.get(nodeId);
+ if (amNode == null) {
+ LOG.info("Ignoring RM Health Update for unknown node: " + nodeId);
+ } else {
+ amNode.handle(rEvent);
+ }
+ break;
+ default:
+ nodeMap.get(nodeId).handle(rEvent);
+ }
+ }
+
+ boolean registerBadNodeAndShouldBlacklist(AMNode amNode) {
+ if (nodeBlacklistingEnabled) {
+ addToBlackList(amNode.getNodeId());
+ computeIgnoreBlacklisting();
+ return !ignoreBlacklisting;
+ } else {
+ return false;
+ }
+ }
+
+ @InterfaceAudience.Private
+ @VisibleForTesting
+ public boolean isBlacklistingIgnored() {
+ return this.ignoreBlacklisting;
+ }
+
+ private void recomputeCurrentIgnoreBlacklistingThreshold() {
+ if (nodeBlacklistingEnabled && blacklistDisablePercent != -1) {
+ currentIgnoreBlacklistingCountThreshold =
+ (float) numClusterNodes * blacklistDisablePercent / 100;
+ }
+ }
+
+ // May be incorrect if there's multiple NodeManagers running on a single host.
+ // knownNodeCount is based on node managers, not hosts. blacklisting is
+ // currently based on hosts.
+ protected void computeIgnoreBlacklisting() {
+
+ boolean stateChanged = false;
+
+ if (!nodeBlacklistingEnabled || blacklistDisablePercent == -1 || blacklistMap.size() == 0) {
+ return;
+ }
+ if (blacklistMap.size() >= currentIgnoreBlacklistingCountThreshold) {
+ if (ignoreBlacklisting == false) {
+ ignoreBlacklisting = true;
+ LOG.info("Ignore Blacklisting set to true. Known: " + numClusterNodes
+ + ", Blacklisted: " + blacklistMap.size());
+ stateChanged = true;
+ }
+ } else {
+ if (ignoreBlacklisting == true) {
+ ignoreBlacklisting = false;
+ LOG.info("Ignore blacklisting set to false. Known: "
+ + numClusterNodes + ", Blacklisted: " + blacklistMap.size());
+ stateChanged = true;
+ }
+ }
+
+ if (stateChanged) {
+ sendIngoreBlacklistingStateToNodes();
+ }
+ }
+
+ private void addToBlackList(NodeId nodeId) {
+ String host = nodeId.getHost();
+
+ if (!blacklistMap.containsKey(host)) {
+ blacklistMap.putIfAbsent(host, new HashSet<NodeId>());
+ }
+ Set<NodeId> nodes = blacklistMap.get(host);
+
+ if (!nodes.contains(nodeId)) {
+ nodes.add(nodeId);
+ }
+ }
+
+ private void sendIngoreBlacklistingStateToNodes() {
+ AMNodeEventType eventType =
+ ignoreBlacklisting ? AMNodeEventType.N_IGNORE_BLACKLISTING_ENABLED
+ : AMNodeEventType.N_IGNORE_BLACKLISTING_DISABLED;
+ for (NodeId nodeId : nodeMap.keySet()) {
+ sendEvent(new AMNodeEvent(nodeId, sourceId, eventType));
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private void sendEvent(Event<?> event) {
+ this.eventHandler.handle(event);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 9882954..0f35bba 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -257,7 +257,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
}
public void preemptContainer(ContainerData cData) {
- getTaskSchedulerEventHandler().containerCompleted(null,
+ getTaskSchedulerEventHandler().containerCompleted(0, null,
ContainerStatus.newInstance(cData.cId, null, "Preempted", ContainerExitStatus.PREEMPTED));
cData.clear();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
index 42d4b0b..7584b4c 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
@@ -200,7 +200,7 @@ public class TestMockDAGAppMaster {
mockLauncher.waitTillContainersLaunched();
ContainerData cData = mockLauncher.getContainers().values().iterator().next();
DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG();
- mockApp.getTaskSchedulerEventHandler().preemptContainer(cData.cId);
+ mockApp.getTaskSchedulerEventHandler().preemptContainer(0, cData.cId);
mockLauncher.startScheduling(true);
dagClient.waitForCompletion();
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 080c20f..62edac9 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -216,9 +216,9 @@ public class TestContainerReuse {
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
verify(taskSchedulerEventHandler).taskAllocated(
- eq(ta11), any(Object.class), eq(containerHost1));
+ eq(0), eq(ta11), any(Object.class), eq(containerHost1));
verify(taskSchedulerEventHandler).taskAllocated(
- eq(ta21), any(Object.class), eq(containerHost2));
+ eq(0), eq(ta21), any(Object.class), eq(containerHost2));
// Adding the event later so that task1 assigned to containerHost1
// is deterministic.
@@ -230,7 +230,7 @@ public class TestContainerReuse {
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler, times(1)).taskAllocated(
- eq(ta31), any(Object.class), eq(containerHost1));
+ eq(0), eq(ta31), any(Object.class), eq(containerHost1));
verify(rmClient, times(0)).releaseAssignedContainer(
eq(containerHost1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -245,7 +245,7 @@ public class TestContainerReuse {
while (System.currentTimeMillis() < currentTs + 5000l) {
try {
verify(taskSchedulerEventHandler,
- times(1)).containerBeingReleased(eq(containerHost2.getId()));
+ times(1)).containerBeingReleased(eq(0), eq(containerHost2.getId()));
exception = null;
break;
} catch (Throwable e) {
@@ -351,8 +351,8 @@ public class TestContainerReuse {
taskScheduler.onContainersAllocated(Lists.newArrayList(containerHost1, containerHost2));
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(containerHost1));
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta21), any(Object.class), eq(containerHost2));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(containerHost1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta21), any(Object.class), eq(containerHost2));
// Adding the event later so that task1 assigned to containerHost1 is deterministic.
taskSchedulerEventHandler.handleEvent(lrTa31);
@@ -363,7 +363,7 @@ public class TestContainerReuse {
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta21), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(
- eq(ta31), any(Object.class), eq(containerHost2));
+ eq(0), eq(ta31), any(Object.class), eq(containerHost2));
verify(rmClient, times(1)).releaseAssignedContainer(
eq(containerHost2.getId()));
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
@@ -459,13 +459,13 @@ public class TestContainerReuse {
taskScheduler.onContainersAllocated(Collections.singletonList(container1));
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1));
// Task assigned to container completed successfully. Container should be re-used.
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta12), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
eventHandler.reset();
@@ -475,7 +475,7 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta12), eq(true), eq((TaskAttemptEndReason)null));
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta13), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
eventHandler.reset();
@@ -483,7 +483,7 @@ public class TestContainerReuse {
// Verify no re-use if a previous task fails.
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, null, 0));
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container1));
verify(taskScheduler).deallocateTask(eq(ta13), eq(false), eq((TaskAttemptEndReason)null));
verify(rmClient).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
@@ -496,7 +496,7 @@ public class TestContainerReuse {
taskScheduler.onContainersAllocated(Collections.singletonList(container2));
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta14), any(Object.class), eq(container2));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container2));
// Task assigned to container completed successfully. No pending requests. Container should be released.
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED, null, 0));
@@ -606,14 +606,14 @@ public class TestContainerReuse {
taskScheduler.onContainersAllocated(Collections.singletonList(container1));
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1));
// First task had profiling on. This container can not be reused further.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
- verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta12), any(Object.class),
+ verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta12), any(Object.class),
eq(container1));
verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
@@ -652,14 +652,14 @@ public class TestContainerReuse {
taskScheduler.onContainersAllocated(Collections.singletonList(container2));
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta13), any(Object.class), eq(container2));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container2));
// Verify that the container can not be reused when profiling option is turned on
// Even for 2 tasks having same profiling option can have container reusability.
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta13), eq(true), eq((TaskAttemptEndReason)null));
- verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class),
+ verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class),
eq(container2));
verify(rmClient, times(1)).releaseAssignedContainer(eq(container2.getId()));
eventHandler.verifyInvocation(AMContainerEventStopRequest.class);
@@ -698,13 +698,13 @@ public class TestContainerReuse {
taskScheduler.onContainersAllocated(Collections.singletonList(container3));
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta15), any(Object.class), eq(container3));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta15), any(Object.class), eq(container3));
//Ensure task 6 (of vertex 1) is allocated to same container
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta15), eq(true), eq((TaskAttemptEndReason)null));
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta16), any(Object.class), eq(container3));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta16), any(Object.class), eq(container3));
eventHandler.reset();
taskScheduler.close();
@@ -804,7 +804,7 @@ public class TestContainerReuse {
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
verify(taskSchedulerEventHandler).taskAllocated(
- eq(ta11), any(Object.class), eq(container1));
+ eq(0), eq(ta11), any(Object.class), eq(container1));
// Send launch request for task2 (vertex2)
taskSchedulerEventHandler.handleEvent(lrEvent12);
@@ -818,7 +818,7 @@ public class TestContainerReuse {
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(
- eq(ta12), any(Object.class), eq(container1));
+ eq(0), eq(ta12), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
eventHandler.reset();
@@ -826,7 +826,7 @@ public class TestContainerReuse {
LOG.info("Sleeping to ensure that the scheduling loop runs");
Thread.sleep(3000l);
verify(taskSchedulerEventHandler).taskAllocated(
- eq(ta12), any(Object.class), eq(container1));
+ eq(0), eq(ta12), any(Object.class), eq(container1));
// TA12 completed.
taskSchedulerEventHandler.handleEvent(
@@ -940,7 +940,7 @@ public class TestContainerReuse {
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
verify(taskSchedulerEventHandler).taskAllocated(
- eq(ta11), any(Object.class), eq(container1));
+ eq(0), eq(ta11), any(Object.class), eq(container1));
// Send launch request for task2 (vertex2)
taskSchedulerEventHandler.handleEvent(lrEvent21);
@@ -953,7 +953,7 @@ public class TestContainerReuse {
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler).taskAllocated(
- eq(ta21), any(Object.class), eq(container1));
+ eq(0), eq(ta21), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
// Task 2 completes.
@@ -1063,7 +1063,7 @@ public class TestContainerReuse {
taskScheduler.onContainersAllocated(Collections.singletonList(container1));
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta111), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta111), any(Object.class), eq(container1));
assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
@@ -1071,7 +1071,7 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
@@ -1114,7 +1114,7 @@ public class TestContainerReuse {
// TODO This is terrible, need a better way to ensure the scheduling loop has run
LOG.info("Sleeping to ensure that the scheduling loop runs");
Thread.sleep(6000l);
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta211), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
@@ -1124,7 +1124,7 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta211), eq(true), eq((TaskAttemptEndReason)null));
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta212), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta212), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
@@ -1237,7 +1237,7 @@ public class TestContainerReuse {
taskScheduler.onContainersAllocated(Collections.singletonList(container1));
TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier);
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta111), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta111), any(Object.class), eq(container1));
assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
@@ -1245,7 +1245,7 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class);
@@ -1290,7 +1290,7 @@ public class TestContainerReuse {
Thread.sleep(6000l);
verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId()));
- verify(taskSchedulerEventHandler).taskAllocated(eq(ta211), any(Object.class), eq(container2));
+ verify(taskSchedulerEventHandler).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container2));
eventHandler.reset();
taskScheduler.close();
@@ -1369,7 +1369,7 @@ public class TestContainerReuse {
drainNotifier.set(false);
taskScheduler.onContainersAllocated(Collections.singletonList(container1));
drainableAppCallback.drain();
- verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta11),
+ verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(0), eq(ta11),
any(Object.class), eq(container1));
taskScheduler.close();
taskSchedulerEventHandler.close();
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index daf1db6..005692e 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -163,7 +163,7 @@ public class TestTaskSchedulerEventHandler {
AMSchedulerEventTALaunchRequest lr =
new AMSchedulerEventTALaunchRequest(mockAttemptId, resource, null, mockTaskAttempt, locHint,
priority, containerContext, 0, 0, 0);
- schedulerHandler.taskAllocated(mockTaskAttempt, lr, container);
+ schedulerHandler.taskAllocated(0, mockTaskAttempt, lr, container);
assertEquals(2, mockEventHandler.events.size());
assertTrue(mockEventHandler.events.get(1) instanceof AMContainerEventAssignTA);
AMContainerEventAssignTA assignEvent =
@@ -227,7 +227,7 @@ public class TestTaskSchedulerEventHandler {
when(mockStatus.getContainerId()).thenReturn(mockCId);
when(mockStatus.getDiagnostics()).thenReturn(diagnostics);
when(mockStatus.getExitStatus()).thenReturn(ContainerExitStatus.PREEMPTED);
- schedulerHandler.containerCompleted(mockTask, mockStatus);
+ schedulerHandler.containerCompleted(0, mockTask, mockStatus);
assertEquals(1, mockEventHandler.events.size());
Event event = mockEventHandler.events.get(0);
assertEquals(AMContainerEventType.C_COMPLETED, event.getType());
@@ -257,7 +257,7 @@ public class TestTaskSchedulerEventHandler {
ContainerId mockCId = mock(ContainerId.class);
verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId)any());
when(mockAMContainerMap.get(mockCId)).thenReturn(mockAmContainer);
- schedulerHandler.preemptContainer(mockCId);
+ schedulerHandler.preemptContainer(0, mockCId);
verify(mockTaskScheduler, times(1)).deallocateContainer(mockCId);
assertEquals(1, mockEventHandler.events.size());
Event event = mockEventHandler.events.get(0);
@@ -290,7 +290,7 @@ public class TestTaskSchedulerEventHandler {
when(mockStatus.getContainerId()).thenReturn(mockCId);
when(mockStatus.getDiagnostics()).thenReturn(diagnostics);
when(mockStatus.getExitStatus()).thenReturn(ContainerExitStatus.DISKS_FAILED);
- schedulerHandler.containerCompleted(mockTask, mockStatus);
+ schedulerHandler.containerCompleted(0, mockTask, mockStatus);
assertEquals(1, mockEventHandler.events.size());
Event event = mockEventHandler.events.get(0);
assertEquals(AMContainerEventType.C_COMPLETED, event.getType());
@@ -325,7 +325,7 @@ public class TestTaskSchedulerEventHandler {
// use -104 rather than ContainerExitStatus.KILLED_EXCEEDED_PMEM because
// ContainerExitStatus.KILLED_EXCEEDED_PMEM is only available after hadoop-2.5
when(mockStatus.getExitStatus()).thenReturn(-104);
- schedulerHandler.containerCompleted(mockTask, mockStatus);
+ schedulerHandler.containerCompleted(0, mockTask, mockStatus);
assertEquals(1, mockEventHandler.events.size());
Event event = mockEventHandler.events.get(0);
assertEquals(AMContainerEventType.C_COMPLETED, event.getType());
@@ -383,4 +383,5 @@ public class TestTaskSchedulerEventHandler {
}
+ // TODO TEZ-2003. Add tests with multiple schedulers, and ensuring that events go out with correct IDs.
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index ffab769..04610ab 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -134,7 +134,7 @@ class TestTaskSchedulerHelpers {
@Override
public void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
- taskSchedulers[0] = new TaskSchedulerWithDrainableAppCallback(this,
+ taskSchedulers[0] = new TaskSchedulerWithDrainableAppCallback(new TaskSchedulerAppCallbackImpl(this, 0),
containerSignatureMatcher, host, port, trackingUrl, amrmClientAsync,
appContext);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8a03f113/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
index d907ea0..84d2e1f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
@@ -93,12 +93,12 @@ public class TestAMNodeTracker {
amNodeTracker.start();
NodeId nodeId = NodeId.newInstance("host1", 2342);
- amNodeTracker.nodeSeen(nodeId);
+ amNodeTracker.nodeSeen(nodeId, 0);
NodeReport nodeReport = generateNodeReport(nodeId, NodeState.UNHEALTHY);
- amNodeTracker.handle(new AMNodeEventStateChanged(nodeReport));
+ amNodeTracker.handle(new AMNodeEventStateChanged(nodeReport, 0));
dispatcher.await();
- assertEquals(AMNodeState.UNHEALTHY, amNodeTracker.get(nodeId).getState());
+ assertEquals(AMNodeState.UNHEALTHY, amNodeTracker.get(nodeId, 0).getState());
amNodeTracker.stop();
}
@@ -114,7 +114,7 @@ public class TestAMNodeTracker {
NodeId nodeId = NodeId.newInstance("unknownhost", 2342);
NodeReport nodeReport = generateNodeReport(nodeId, NodeState.UNHEALTHY);
- amNodeTracker.handle(new AMNodeEventStateChanged(nodeReport));
+ amNodeTracker.handle(new AMNodeEventStateChanged(nodeReport, 0));
dispatcher.await();
amNodeTracker.stop();
@@ -142,27 +142,27 @@ public class TestAMNodeTracker {
amNodeTracker.init(conf);
amNodeTracker.start();
- amNodeTracker.handle(new AMNodeEventNodeCountUpdated(1));
+ amNodeTracker.handle(new AMNodeEventNodeCountUpdated(1, 0));
NodeId nodeId = NodeId.newInstance("host1", 1234);
- amNodeTracker.nodeSeen(nodeId);
+ amNodeTracker.nodeSeen(nodeId, 0);
- AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId);
+ AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, 0);
ContainerId cId1 = mock(ContainerId.class);
ContainerId cId2 = mock(ContainerId.class);
- amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId1));
- amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId2));
+ amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId1));
+ amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId2));
TezTaskAttemptID ta1 = mock(TezTaskAttemptID.class);
TezTaskAttemptID ta2 = mock(TezTaskAttemptID.class);
- amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId1, ta1, true));
+ amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId1, ta1, true));
dispatcher.await();
assertEquals(1, node.numFailedTAs);
assertEquals(AMNodeState.ACTIVE, node.getState());
- amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId2, ta2, true));
+ amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
dispatcher.await();
assertEquals(2, node.numFailedTAs);
assertEquals(1, handler.events.size());
@@ -187,44 +187,44 @@ public class TestAMNodeTracker {
amNodeTracker.init(conf);
amNodeTracker.start();
- amNodeTracker.handle(new AMNodeEventNodeCountUpdated(4));
+ amNodeTracker.handle(new AMNodeEventNodeCountUpdated(4, 0));
NodeId nodeId = NodeId.newInstance("host1", 1234);
NodeId nodeId2 = NodeId.newInstance("host2", 1234);
NodeId nodeId3 = NodeId.newInstance("host3", 1234);
NodeId nodeId4 = NodeId.newInstance("host4", 1234);
- amNodeTracker.nodeSeen(nodeId);
- amNodeTracker.nodeSeen(nodeId2);
- amNodeTracker.nodeSeen(nodeId3);
- amNodeTracker.nodeSeen(nodeId4);
- AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId);
+ amNodeTracker.nodeSeen(nodeId, 0);
+ amNodeTracker.nodeSeen(nodeId2, 0);
+ amNodeTracker.nodeSeen(nodeId3, 0);
+ amNodeTracker.nodeSeen(nodeId4, 0);
+ AMNodeImpl node = (AMNodeImpl) amNodeTracker.get(nodeId, 0);
ContainerId cId1 = mock(ContainerId.class);
ContainerId cId2 = mock(ContainerId.class);
ContainerId cId3 = mock(ContainerId.class);
- amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId1));
- amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId2));
- amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, cId3));
+ amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId1));
+ amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId2));
+ amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId, 0, cId3));
assertEquals(3, node.containers.size());
TezTaskAttemptID ta1 = mock(TezTaskAttemptID.class);
TezTaskAttemptID ta2 = mock(TezTaskAttemptID.class);
TezTaskAttemptID ta3 = mock(TezTaskAttemptID.class);
- amNodeTracker.handle(new AMNodeEventTaskAttemptSucceeded(nodeId, cId1, ta1));
+ amNodeTracker.handle(new AMNodeEventTaskAttemptSucceeded(nodeId, 0, cId1, ta1));
assertEquals(1, node.numSuccessfulTAs);
- amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId2, ta2, true));
+ amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
assertEquals(1, node.numSuccessfulTAs);
assertEquals(1, node.numFailedTAs);
assertEquals(AMNodeState.ACTIVE, node.getState());
// duplicate should not affect anything
- amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId2, ta2, true));
+ amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId2, ta2, true));
assertEquals(1, node.numSuccessfulTAs);
assertEquals(1, node.numFailedTAs);
assertEquals(AMNodeState.ACTIVE, node.getState());
- amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, cId3, ta3, true));
+ amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId, 0, cId3, ta3, true));
dispatcher.await();
assertEquals(1, node.numSuccessfulTAs);
assertEquals(2, node.numFailedTAs);
@@ -246,20 +246,20 @@ public class TestAMNodeTracker {
ContainerId cId5 = mock(ContainerId.class);
TezTaskAttemptID ta4 = mock(TezTaskAttemptID.class);
TezTaskAttemptID ta5 = mock(TezTaskAttemptID.class);
- AMNodeImpl node2 = (AMNodeImpl) amNodeTracker.get(nodeId2);
- amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, cId4));
- amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, cId5));
+ AMNodeImpl node2 = (AMNodeImpl) amNodeTracker.get(nodeId2, 0);
+ amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, 0, cId4));
+ amNodeTracker.handle(new AMNodeEventContainerAllocated(nodeId2, 0, cId5));
- amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, cId4, ta4, true));
+ amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, 0, cId4, ta4, true));
assertEquals(1, node2.numFailedTAs);
assertEquals(AMNodeState.ACTIVE, node2.getState());
handler.events.clear();
- amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, cId5, ta5, true));
+ amNodeTracker.handle(new AMNodeEventTaskAttemptEnded(nodeId2, 0, cId5, ta5, true));
dispatcher.await();
assertEquals(2, node2.numFailedTAs);
assertEquals(AMNodeState.FORCED_ACTIVE, node2.getState());
- AMNodeImpl node3 = (AMNodeImpl) amNodeTracker.get(nodeId3);
+ AMNodeImpl node3 = (AMNodeImpl) amNodeTracker.get(nodeId3, 0);
assertEquals(AMNodeState.FORCED_ACTIVE, node3.getState());
assertEquals(5, handler.events.size());
@@ -286,7 +286,7 @@ public class TestAMNodeTracker {
// Increase the number of nodes. BLACKLISTING should be re-enabled.
// Node 1 and Node 2 should go into BLACKLISTED state
handler.events.clear();
- amNodeTracker.handle(new AMNodeEventNodeCountUpdated(8));
+ amNodeTracker.handle(new AMNodeEventNodeCountUpdated(8, 0));
dispatcher.await();
LOG.info(("Completed waiting for dispatcher to process all pending events"));
assertEquals(AMNodeState.BLACKLISTED, node.getState());
@@ -336,4 +336,6 @@ public class TestAMNodeTracker {
doReturn(healthReportTime).when(nodeReport).getLastHealthReportTime();
return nodeReport;
}
+
+ // TODO TEZ-2003. Add tests for multiple sources.
}
[19/51] [abbrv] tez git commit: TEZ-2347. Expose additional
information in TaskCommunicatorContext. (sseth)
Posted by ss...@apache.org.
TEZ-2347. Expose additional information in TaskCommunicatorContext. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/e8c0c7a8
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/e8c0c7a8
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/e8c0c7a8
Branch: refs/heads/TEZ-2003
Commit: e8c0c7a8a635903dd5809807cc94469ff8b3e88a
Parents: 07e5e42
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 20 13:17:31 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../tez/dag/api/TaskCommunicatorContext.java | 50 ++++++++++++++++++++
.../dag/app/TaskCommunicatorContextImpl.java | 50 ++++++++++++++++++++
.../java/org/apache/tez/dag/app/dag/DAG.java | 2 +
.../java/org/apache/tez/dag/app/dag/Task.java | 2 +
.../org/apache/tez/dag/app/dag/TaskAttempt.java | 6 +++
.../apache/tez/dag/app/dag/impl/DAGImpl.java | 10 ++++
.../tez/dag/app/dag/impl/TaskAttemptImpl.java | 12 +++++
.../apache/tez/dag/app/dag/impl/TaskImpl.java | 13 ++++-
9 files changed, 145 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index ca5225e..7c13110 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -16,5 +16,6 @@ ALL CHANGES:
TEZ-2284. Separate TaskReporter into an interface.
TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
+ TEZ-2347. Expose additional information in TaskCommunicatorContext.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 19caed9..56345ab 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -16,6 +16,7 @@ package org.apache.tez.dag.api;
import javax.annotation.Nullable;
import java.io.IOException;
+import java.util.Collection;
import java.util.Set;
import org.apache.hadoop.security.Credentials;
@@ -71,4 +72,53 @@ public interface TaskCommunicatorContext {
// TODO TEZ-2003 API. Should a method exist for task succeeded.
// TODO Eventually Add methods to report availability stats to the scheduler.
+
+ /**
+ * Get the name of the currently executing dag
+ * @return the name of the currently executing dag
+ */
+ String getCurretnDagName();
+
+ /**
+ * Get the name of the Input vertices for the specified vertex.
+ * Root Inputs are not returned.
+ * @param vertexName the vertex for which source vertex names will be returned
+ * @return an Iterable containing the list of input vertices for the specified vertex
+ */
+ Iterable<String> getInputVertexNames(String vertexName);
+
+ /**
+ * Get the total number of tasks in the given vertex
+ * @param vertexName
+ * @return total number of tasks in this vertex
+ */
+ int getVertexTotalTaskCount(String vertexName);
+
+ /**
+ * Get the number of completed tasks for a given vertex
+ * @param vertexName the vertex name
+ * @return the number of completed tasks for the vertex
+ */
+ int getVertexCompletedTaskCount(String vertexName);
+
+ /**
+ * Get the number of running tasks for a given vertex
+ * @param vertexName the vertex name
+ * @return the number of running tasks for the vertex
+ */
+ int getVertexRunningTaskCount(String vertexName);
+
+ /**
+ * Get the start time for the first attempt of the specified task
+ * @param vertexName the vertex to which the task belongs
+ * @param taskIndex the index of the task
+ * @return the start time for the first attempt of the task
+ */
+ long getFirstAttemptStartTime(String vertexName, int taskIndex);
+
+ /**
+ * Get the start time for the currently executing DAG
+ * @return time when the current dag started executing
+ */
+ long getDagStartTime();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index 3714c3c..4cb0c93 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -18,7 +18,9 @@ import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Set;
+import com.google.common.base.Function;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -31,6 +33,7 @@ import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.event.VertexState;
import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.app.dag.Vertex;
import org.apache.tez.dag.app.dag.VertexStateUpdateListener;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -111,6 +114,53 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
context.getCurrentDAG().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet, this);
}
+ @Override
+ public String getCurretnDagName() {
+ return context.getCurrentDAG().getName();
+ }
+
+ @Override
+ public Iterable<String> getInputVertexNames(String vertexName) {
+ Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
+ Vertex vertex = context.getCurrentDAG().getVertex(vertexName);
+ Set<Vertex> sources = vertex.getInputVertices().keySet();
+ return Iterables.transform(sources, new Function<Vertex, String>() {
+ @Override
+ public String apply(@Nullable Vertex input) {
+ return input.getName();
+ }
+ });
+ }
+
+ @Override
+ public int getVertexTotalTaskCount(String vertexName) {
+ Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
+ return context.getCurrentDAG().getVertex(vertexName).getTotalTasks();
+ }
+
+ @Override
+ public int getVertexCompletedTaskCount(String vertexName) {
+ Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
+ return context.getCurrentDAG().getVertex(vertexName).getCompletedTasks();
+ }
+
+ @Override
+ public int getVertexRunningTaskCount(String vertexName) {
+ Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
+ return context.getCurrentDAG().getVertex(vertexName).getRunningTasks();
+ }
+
+ @Override
+ public long getFirstAttemptStartTime(String vertexName, int taskIndex) {
+ Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
+ Preconditions.checkArgument(taskIndex >=0, "TaskIndex must be > 0");
+ return context.getCurrentDAG().getVertex(vertexName).getTask(taskIndex).getFirstAttemptStartTime();
+ }
+
+ @Override
+ public long getDagStartTime() {
+ return context.getCurrentDAG().getStartTime();
+ }
@Override
public void onStateUpdated(VertexStateUpdate event) {
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
index 6d6872b..458362f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
@@ -94,6 +94,8 @@ public interface DAG {
Map<String, TezVertexID> getVertexNameIDMapping();
+ long getStartTime();
+
StateChangeNotifier getStateChangeNotifier();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
index 47b56f2..a011b61 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
@@ -72,4 +72,6 @@ public interface Task {
public TaskSpec getBaseTaskSpec();
public TaskLocationHint getTaskLocationHint();
+
+ long getFirstAttemptStartTime();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
index 4360cc3..cbe72c1 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
@@ -125,6 +125,12 @@ public interface TaskAttempt {
*/
long getLaunchTime();
+ /**
+ * Get the time at which this attempt was scheduled
+ * @return the time at which this attempt was scheduled, 0 if it hasn't been scheduled yet
+ */
+ long getScheduleTime();
+
/**
* @return attempt's finish time. If attempt is not finished
* yet, returns 0.
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index ef2df78..e37fc2f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -702,6 +702,16 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
}
@Override
+ public long getStartTime() {
+ readLock.lock();
+ try {
+ return this.startTime;
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ @Override
public StateChangeNotifier getStateChangeNotifier() {
return entityUpdateTracker;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
index e70123e..5085691 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -133,6 +133,7 @@ public class TaskAttemptImpl implements TaskAttempt,
protected final AppContext appContext;
private final TaskHeartbeatHandler taskHeartbeatHandler;
private long launchTime = 0;
+ private long scheduleTime = 0;
private long finishTime = 0;
private String trackerName;
private int httpPort;
@@ -670,6 +671,16 @@ public class TaskAttemptImpl implements TaskAttempt,
}
@Override
+ public long getScheduleTime() {
+ readLock.lock();
+ try {
+ return scheduleTime;
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ @Override
public long getFinishTime() {
readLock.lock();
try {
@@ -1029,6 +1040,7 @@ public class TaskAttemptImpl implements TaskAttempt,
public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
TaskAttemptEventSchedule scheduleEvent = (TaskAttemptEventSchedule) event;
+ ta.scheduleTime = ta.clock.getTime();
// TODO Creating the remote task here may not be required in case of
// recovery.
http://git-wip-us.apache.org/repos/asf/tez/blob/e8c0c7a8/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
index e6027f5..93b4c3f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
@@ -1529,7 +1529,18 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
this.writeLock.unlock();
}
}
-
+
+ @Override
+ public long getFirstAttemptStartTime() {
+ readLock.lock();
+ try {
+ // The first attempt will always have an index of 0.
+ return getAttempt(TezTaskAttemptID.getInstance(getTaskId(), 0)).getScheduleTime();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
private static class KillTransition
implements SingleArcTransition<TaskImpl, TaskEvent> {
@Override
[05/51] [abbrv] tez git commit: TEZ-2122. Setup pluggable components
at AM/Vertex level. (sseth)
Posted by ss...@apache.org.
TEZ-2122. Setup pluggable components at AM/Vertex level. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/b0568218
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/b0568218
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/b0568218
Branch: refs/heads/TEZ-2003
Commit: b0568218d45e757fdc2bc2e56396e47fbe9697fc
Parents: e025255
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Feb 19 14:59:18 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:24:03 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../apache/tez/dag/api/TezConfiguration.java | 29 +++-
.../org/apache/tez/dag/api/TezConstants.java | 3 +
.../java/org/apache/tez/dag/app/AppContext.java | 4 +
.../org/apache/tez/dag/app/DAGAppMaster.java | 121 +++++++++++++-
.../dag/app/TaskAttemptListenerImpTezDag.java | 77 +++++----
.../java/org/apache/tez/dag/app/dag/Vertex.java | 4 +
.../tez/dag/app/dag/impl/TaskAttemptImpl.java | 8 +-
.../apache/tez/dag/app/dag/impl/VertexImpl.java | 47 ++++++
.../app/launcher/ContainerLauncherRouter.java | 93 +++++++----
.../app/rm/AMSchedulerEventTALaunchRequest.java | 22 ++-
.../dag/app/rm/TaskSchedulerEventHandler.java | 163 +++++++++++--------
.../apache/tez/dag/app/MockDAGAppMaster.java | 5 +-
.../app/TestTaskAttemptListenerImplTezDag.java | 10 +-
.../tez/dag/app/rm/TestContainerReuse.java | 2 +-
.../app/rm/TestTaskSchedulerEventHandler.java | 12 +-
.../dag/app/rm/TestTaskSchedulerHelpers.java | 18 +-
.../tez/tests/TestExternalTezServices.java | 19 ++-
18 files changed, 458 insertions(+), 180 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 1cd74a4..4bfe08f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -3,5 +3,6 @@ ALL CHANGES:
TEZ-2006. Task communication plane needs to be pluggable.
TEZ-2090. Add tests for jobs running in external services.
TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
+ TEZ-2122. Setup pluggable components at AM/Vertex level.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 34fdb15..39a4c77 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -1215,13 +1215,36 @@ public class TezConfiguration extends Configuration {
+ "tez-ui.webservice.enable";
public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true;
+ /** defaults container-launcher for the specific vertex */
@ConfigurationScope(Scope.VERTEX)
- public static final String TEZ_AM_CONTAINER_LAUNCHER_CLASS = TEZ_AM_PREFIX + "container-launcher.class";
+ public static final String TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME = TEZ_AM_PREFIX + "vertex.container-launcher.name";
+ /** defaults task-scheduler for the specific vertex */
@ConfigurationScope(Scope.VERTEX)
- public static final String TEZ_AM_TASK_SCHEDULER_CLASS = TEZ_AM_PREFIX + "task-scheduler.class";
+ public static final String TEZ_AM_VERTEX_TASK_SCHEDULER_NAME = TEZ_AM_PREFIX + "vertex.task-scheduler.name";
+ /** defaults task-communicator for the specific vertex */
@ConfigurationScope(Scope.VERTEX)
- public static final String TEZ_AM_TASK_COMMUNICATOR_CLASS = TEZ_AM_PREFIX + "task-communicator.class";
+ public static final String TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME = TEZ_AM_PREFIX + "vertex.task-communicator.name";
+ /** Comma separated list of named container-launcher classes running in the AM.
+ * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
+ * e.g. Tez, ExtService:org.apache.ExtLauncherClasss
+ * */
+ @ConfigurationScope(Scope.AM)
+ public static final String TEZ_AM_CONTAINER_LAUNCHERS = TEZ_AM_PREFIX + "container-launchers";
+
+ /** Comma separated list of task-schedulers classes running in the AM.
+ * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
+ * e.g. Tez, ExtService:org.apache.ExtSchedulerClasss
+ */
+ @ConfigurationScope(Scope.AM)
+ public static final String TEZ_AM_TASK_SCHEDULERS = TEZ_AM_PREFIX + "task-schedulers";
+
+ /** Comma separated list of task-communicators classes running in the AM.
+ * The format for each entry is NAME:CLASSNAME, except for tez default which is specified as Tez
+ * e.g. Tez, ExtService:org.apache.ExtTaskCommClass
+ * */
+ @ConfigurationScope(Scope.AM)
+ public static final String TEZ_AM_TASK_COMMUNICATORS = TEZ_AM_PREFIX + "task-communicators";
// TODO only validate property here, value can also be validated if necessary
public static void validateProperty(String property, Scope usedScope) {
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
index bc4208f..3b07c59 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
@@ -102,4 +102,7 @@ public class TezConstants {
/// Version-related Environment variables
public static final String TEZ_CLIENT_VERSION_ENV = "TEZ_CLIENT_VERSION";
+
+ public static final String TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT = "Tez";
+ public static final String TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT = "TezLocal";
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
index bf3e318..1ccb10b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
@@ -114,4 +114,8 @@ public interface AppContext {
boolean isAMInCompletionState();
Credentials getAppCredentials();
+
+ public Integer getTaskCommunicatorIdentifier(String name);
+ public Integer getTaskScheduerIdentifier(String name);
+ public Integer getContainerLauncherIdentifier(String name);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 824dfbc..431a8b2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -56,6 +56,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.Options;
@@ -270,7 +272,12 @@ public class DAGAppMaster extends AbstractService {
private ExecutorService rawExecutor;
private ListeningExecutorService execService;
-
+
+ // TODO May not need to be a bidi map
+ private final BiMap<String, Integer> taskSchedulers = HashBiMap.create();
+ private final BiMap<String, Integer> containerLaunchers = HashBiMap.create();
+ private final BiMap<String, Integer> taskCommunicators = HashBiMap.create();
+
/**
* set of already executed dag names.
*/
@@ -374,6 +381,29 @@ public class DAGAppMaster extends AbstractService {
this.isLocal = conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
+ String tezDefaultClassIdentifier =
+ isLocal ? TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT :
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
+
+ String[] taskSchedulerClassIdentifiers = parsePlugins(taskSchedulers,
+ conf.getTrimmedStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
+ tezDefaultClassIdentifier),
+ TezConfiguration.TEZ_AM_TASK_SCHEDULERS);
+
+ String[] containerLauncherClassIdentifiers = parsePlugins(containerLaunchers,
+ conf.getTrimmedStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
+ tezDefaultClassIdentifier),
+ TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS);
+
+ String[] taskCommunicatorClassIdentifiers = parsePlugins(taskCommunicators,
+ conf.getTrimmedStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
+ tezDefaultClassIdentifier),
+ TezConfiguration.TEZ_AM_TASK_COMMUNICATORS);
+
+ LOG.info(buildPluginComponentLog(taskSchedulerClassIdentifiers, taskSchedulers, "TaskSchedulers"));
+ LOG.info(buildPluginComponentLog(containerLauncherClassIdentifiers, containerLaunchers, "ContainerLaunchers"));
+ LOG.info(buildPluginComponentLog(taskCommunicatorClassIdentifiers, taskCommunicators, "TaskCommunicators"));
+
boolean disableVersionCheck = conf.getBoolean(
TezConfiguration.TEZ_AM_DISABLE_CLIENT_VERSION_CHECK,
TezConfiguration.TEZ_AM_DISABLE_CLIENT_VERSION_CHECK_DEFAULT);
@@ -439,7 +469,7 @@ public class DAGAppMaster extends AbstractService {
//service to handle requests to TaskUmbilicalProtocol
taskAttemptListener = createTaskAttemptListener(context,
- taskHeartbeatHandler, containerHeartbeatHandler);
+ taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorClassIdentifiers);
addIfService(taskAttemptListener, true);
containerSignatureMatcher = createContainerSignatureMatcher();
@@ -486,7 +516,8 @@ public class DAGAppMaster extends AbstractService {
}
this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
- clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService);
+ clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
+ taskSchedulerClassIdentifiers);
addIfService(taskSchedulerEventHandler, true);
if (enableWebUIService()) {
@@ -504,7 +535,7 @@ public class DAGAppMaster extends AbstractService {
taskSchedulerEventHandler);
addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
- this.containerLauncherRouter = createContainerLauncherRouter(conf);
+ this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherClassIdentifiers);
addIfService(containerLauncherRouter, true);
dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
@@ -1012,9 +1043,9 @@ public class DAGAppMaster extends AbstractService {
}
protected TaskAttemptListener createTaskAttemptListener(AppContext context,
- TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh) {
+ TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh, String[] taskCommunicatorClasses) {
TaskAttemptListener lis =
- new TaskAttemptListenerImpTezDag(context, thh, chh,jobTokenSecretManager);
+ new TaskAttemptListenerImpTezDag(context, thh, chh,jobTokenSecretManager, taskCommunicatorClasses);
return lis;
}
@@ -1035,9 +1066,9 @@ public class DAGAppMaster extends AbstractService {
return chh;
}
- protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf) throws
+ protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf, String []containerLauncherClasses) throws
UnknownHostException {
- return new ContainerLauncherRouter(conf, isLocal, context, taskAttemptListener, workingDirectory);
+ return new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory, containerLauncherClasses);
}
@@ -1499,6 +1530,21 @@ public class DAGAppMaster extends AbstractService {
}
@Override
+ public Integer getTaskCommunicatorIdentifier(String name) {
+ return taskCommunicators.get(name);
+ }
+
+ @Override
+ public Integer getTaskScheduerIdentifier(String name) {
+ return taskSchedulers.get(name);
+ }
+
+ @Override
+ public Integer getContainerLauncherIdentifier(String name) {
+ return taskCommunicators.get(name);
+ }
+
+ @Override
public Map<ApplicationAccessType, String> getApplicationACLs() {
if (getServiceState() != STATE.STARTED) {
throw new TezUncheckedException(
@@ -2291,4 +2337,63 @@ public class DAGAppMaster extends AbstractService {
return amConf.getBoolean(TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE,
TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT);
}
+
+ // Tez default classnames are populated as TezConfiguration.TEZ_AM_SERVICE_PLUGINS_DEFAULT
+ private String[] parsePlugins(BiMap<String, Integer> pluginMap, String[] pluginStrings,
+ String context) {
+ Preconditions.checkState(pluginStrings != null && pluginStrings.length > 0,
+ "Plugin strings should not be null or empty: " + context);
+
+ String[] classNames = new String[pluginStrings.length];
+
+ int index = 0;
+ for (String pluginString : pluginStrings) {
+
+ String className;
+ String identifierString;
+
+ Preconditions.checkState(pluginString != null && !pluginString.isEmpty(),
+ "Plugin string: " + pluginString + " should not be null or empty");
+ if (pluginString.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) ||
+ pluginString.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+ // Kind of ugly, but Tez internal routing is encoded via a String instead of classnames.
+ // Individual components - TaskComm, Scheduler, Launcher deal with actual classname translation,
+ // and avoid reflection.
+ identifierString = pluginString;
+ className = pluginString;
+ } else {
+ String[] parts = pluginString.split(":");
+ Preconditions.checkState(
+ parts.length == 2 && parts[0] != null && !parts[0].isEmpty() && parts[1] != null &&
+ !parts[1].isEmpty(),
+ "Invalid configuration string for " + context + ": " + pluginString);
+ Preconditions.checkState(
+ !parts[0].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) &&
+ !parts[0].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT),
+ "Identifier cannot be " + TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT + " or " +
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT + " for " +
+ pluginString);
+ identifierString = parts[0];
+ className = parts[1];
+ }
+ pluginMap.put(identifierString, index);
+ classNames[index] = className;
+ }
+ return classNames;
+ }
+
+ String buildPluginComponentLog(String[] classIdentifiers, BiMap<String, Integer> map,
+ String component) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("AM Level configured ").append(component).append(": ");
+ for (int i = 0; i < classIdentifiers.length; i++) {
+ sb.append("[").append(i).append(":").append(map.inverse().get(i)).append(":")
+ .append(taskSchedulers.inverse().get(i)).append(
+ "]");
+ if (i != classIdentifiers.length - 1) {
+ sb.append(",");
+ }
+ }
+ return sb.toString();
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index ff50907..3d9abdf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -37,7 +37,6 @@ import org.apache.tez.runtime.api.impl.EventType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -46,7 +45,7 @@ import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
-import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -56,7 +55,6 @@ import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
-import org.apache.tez.dag.app.rm.TaskSchedulerService;
import org.apache.tez.dag.app.rm.container.AMContainerTask;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezVertexID;
@@ -73,7 +71,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
.getLogger(TaskAttemptListenerImpTezDag.class);
private final AppContext context;
- private TaskCommunicator taskCommunicator;
+ private final TaskCommunicator[] taskCommunicators;
protected final TaskHeartbeatHandler taskHeartbeatHandler;
protected final ContainerHeartbeatHandler containerHeartbeatHandler;
@@ -99,28 +97,52 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
public TaskAttemptListenerImpTezDag(AppContext context,
TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
// TODO TEZ-2003 pre-merge. Remove reference to JobTokenSecretManager.
- JobTokenSecretManager jobTokenSecretManager) {
+ JobTokenSecretManager jobTokenSecretManager,
+ String [] taskCommunicatorClassIdentifiers) {
super(TaskAttemptListenerImpTezDag.class.getName());
this.context = context;
this.taskHeartbeatHandler = thh;
this.containerHeartbeatHandler = chh;
- this.taskCommunicator = new TezTaskCommunicatorImpl(this);
+ if (taskCommunicatorClassIdentifiers == null || taskCommunicatorClassIdentifiers.length == 0) {
+ taskCommunicatorClassIdentifiers = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ }
+ this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
+ for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
+ taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i]);
+ }
+ // TODO TEZ-2118 Start using taskCommunicator indices properly
+ }
+
+ @Override
+ public void serviceStart() {
+ // TODO Why is init tied to serviceStart
+ for (int i = 0 ; i < taskCommunicators.length ; i++) {
+ taskCommunicators[i].init(getConfig());
+ taskCommunicators[i].start();
+ }
}
@Override
- public void serviceInit(Configuration conf) {
- String taskCommClassName = conf.get(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS);
- if (taskCommClassName == null) {
+ public void serviceStop() {
+ for (int i = 0 ; i < taskCommunicators.length ; i++) {
+ taskCommunicators[i].stop();
+ }
+ }
+
+ private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier) {
+ if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT) ||
+ taskCommClassIdentifier
+ .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Using Default Task Communicator");
- this.taskCommunicator = new TezTaskCommunicatorImpl(this);
+ return new TezTaskCommunicatorImpl(this);
} else {
- LOG.info("Using TaskCommunicator: " + taskCommClassName);
+ LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
- .getClazz(taskCommClassName);
+ .getClazz(taskCommClassIdentifier);
try {
Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
ctor.setAccessible(true);
- this.taskCommunicator = ctor.newInstance(this);
+ return ctor.newInstance(this);
} catch (NoSuchMethodException e) {
throw new TezUncheckedException(e);
} catch (InvocationTargetException e) {
@@ -134,20 +156,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
@Override
- public void serviceStart() {
- taskCommunicator.init(getConfig());
- taskCommunicator.start();
- }
-
- @Override
- public void serviceStop() {
- if (taskCommunicator != null) {
- taskCommunicator.stop();
- taskCommunicator = null;
- }
- }
-
- @Override
public ApplicationAttemptId getApplicationAttemptId() {
return context.getApplicationAttemptId();
}
@@ -235,7 +243,8 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
@Override
public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
- context.getEventHandler().handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
+ context.getEventHandler()
+ .handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
pingContainerHeartbeatHandler(containerId);
}
@@ -265,7 +274,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
@Override
public InetSocketAddress getAddress() {
- return taskCommunicator.getAddress();
+ return taskCommunicators[0].getAddress();
}
// The TaskAttemptListener register / unregister methods in this class are not thread safe.
@@ -297,7 +306,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
"Multiple registrations for containerId: " + containerId);
}
NodeId nodeId = context.getAllContainers().get(containerId).getContainer().getNodeId();
- taskCommunicator.registerRunningContainer(containerId, nodeId.getHost(), nodeId.getPort());
+ taskCommunicators[0].registerRunningContainer(containerId, nodeId.getHost(), nodeId.getPort());
}
@Override
@@ -309,7 +318,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
if (containerInfo.taskAttemptId != null) {
registeredAttempts.remove(containerInfo.taskAttemptId);
}
- taskCommunicator.registerContainerEnd(containerId);
+ taskCommunicators[0].registerContainerEnd(containerId);
}
@Override
@@ -344,7 +353,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
+ amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+ " when already assigned to: " + containerIdFromMap);
}
- taskCommunicator.registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
+ taskCommunicators[0].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
amContainerTask.haveCredentialsChanged());
}
@@ -364,7 +373,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
// Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
registeredContainers.put(containerId, NULL_CONTAINER_INFO);
- taskCommunicator.unregisterRunningTaskAttempt(attemptId);
+ taskCommunicators[0].unregisterRunningTaskAttempt(attemptId);
}
private void pingContainerHeartbeatHandler(ContainerId containerId) {
@@ -383,6 +392,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
public TaskCommunicator getTaskCommunicator() {
- return taskCommunicator;
+ return taskCommunicators[0];
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
index ab7941e..552da11 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
@@ -173,4 +173,8 @@ public interface Vertex extends Comparable<Vertex> {
public int getKilledTaskAttemptCount();
public Configuration getConf();
+
+ public int getTaskSchedulerIdentifier();
+ public int getContainerLauncherIdentifier();
+ public int getTaskCommunicatorIdentifier();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
index ebf7c58..806b977 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -1065,9 +1065,15 @@ public class TaskAttemptImpl implements TaskAttempt,
priority = (scheduleEvent.getPriorityHighLimit() + scheduleEvent.getPriorityLowLimit()) / 2;
}
+ // TODO Jira post TEZ-2003 getVertex implementation is very inefficient. This should be via references, instead of locked table lookups.
+ Vertex vertex = ta.getVertex();
AMSchedulerEventTALaunchRequest launchRequestEvent = new AMSchedulerEventTALaunchRequest(
ta.attemptId, ta.taskResource, remoteTaskSpec, ta, locationHint,
- priority, ta.containerContext);
+ priority, ta.containerContext,
+ vertex.getTaskSchedulerIdentifier(),
+ vertex.getContainerLauncherIdentifier(),
+ vertex.getTaskCommunicatorIdentifier());
+
ta.sendEvent(launchRequestEvent);
return TaskAttemptStateInternal.START_WAIT;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index 1fcfe7e..0583a0b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -74,6 +74,7 @@ import org.apache.tez.dag.api.ProcessorDescriptor;
import org.apache.tez.dag.api.RootInputLeafOutput;
import org.apache.tez.dag.api.Scope;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.VertexLocationHint;
import org.apache.tez.dag.api.TaskLocationHint;
@@ -232,6 +233,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
private final boolean isSpeculationEnabled;
+ private final int taskSchedulerIdentifier;
+ private final int containerLauncherIdentifier;
+ private final int taskCommunicatorIdentifier;
+
//fields initialized in init
@VisibleForTesting
@@ -986,6 +991,33 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
// This "this leak" is okay because the retained pointer is in an
// instance variable.
+ boolean isLocal = vertexConf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
+ TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
+
+ String tezDefaultComponentName =
+ isLocal ? TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT :
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
+ String taskSchedulerName =
+ vertexConf.get(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, tezDefaultComponentName);
+ String taskCommName = vertexConf
+ .get(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, tezDefaultComponentName);
+ String containerLauncherName = vertexConf
+ .get(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, tezDefaultComponentName);
+ taskSchedulerIdentifier = appContext.getTaskScheduerIdentifier(taskSchedulerName);
+ taskCommunicatorIdentifier = appContext.getTaskCommunicatorIdentifier(taskCommName);
+ containerLauncherIdentifier = appContext.getContainerLauncherIdentifier(containerLauncherName);
+
+ Preconditions.checkNotNull(taskSchedulerIdentifier, "Unknown taskScheduler: " + taskSchedulerName);
+ Preconditions.checkNotNull(taskCommunicatorIdentifier, "Unknown taskCommunicator: " + containerLauncherName);
+ Preconditions.checkNotNull(containerLauncherIdentifier, "Unknown containerLauncher: " + taskCommName);
+
+ StringBuilder sb = new StringBuilder();
+ sb.append("Running vertex: ").append(logIdentifier).append(" : ")
+ .append("TaskScheduler=").append(taskSchedulerIdentifier).append(":").append(taskSchedulerName)
+ .append(", ContainerLauncher=").append(containerLauncherIdentifier).append(":").append(containerLauncherName)
+ .append(", TaskCommunicator=").append(taskCommunicatorIdentifier).append(":").append(taskCommName);
+ LOG.info(sb.toString());
+
stateMachine = new StateMachineTez<VertexState, VertexEventType, VertexEvent, VertexImpl>(
stateMachineFactory.make(this), this);
augmentStateMachine();
@@ -996,6 +1028,21 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
return vertexConf;
}
+ @Override
+ public int getTaskSchedulerIdentifier() {
+ return this.taskSchedulerIdentifier;
+ }
+
+ @Override
+ public int getContainerLauncherIdentifier() {
+ return this.containerLauncherIdentifier;
+ }
+
+ @Override
+ public int getTaskCommunicatorIdentifier() {
+ return this.taskCommunicatorIdentifier;
+ }
+
private boolean isSpeculationEnabled() {
return isSpeculationEnabled;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 34001ed..621e4a8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.TaskAttemptListener;
@@ -36,73 +37,93 @@ public class ContainerLauncherRouter extends AbstractService
static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
- private final ContainerLauncher containerLauncher;
+ private final ContainerLauncher containerLaunchers[];
@VisibleForTesting
public ContainerLauncherRouter(ContainerLauncher containerLauncher) {
super(ContainerLauncherRouter.class.getName());
- this.containerLauncher = containerLauncher;
+ containerLaunchers = new ContainerLauncher[] {containerLauncher};
}
// Accepting conf to setup final parameters, if required.
- public ContainerLauncherRouter(Configuration conf, boolean isLocal, AppContext context,
+ public ContainerLauncherRouter(Configuration conf, AppContext context,
TaskAttemptListener taskAttemptListener,
- String workingDirectory) throws UnknownHostException {
+ String workingDirectory,
+ String[] containerLauncherClassIdentifiers) throws UnknownHostException {
super(ContainerLauncherRouter.class.getName());
- if (isLocal) {
+ if (containerLauncherClassIdentifiers == null || containerLauncherClassIdentifiers.length == 0) {
+ containerLauncherClassIdentifiers = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ }
+ containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
+
+ for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
+ containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
+ taskAttemptListener, workingDirectory, conf);
+ }
+ }
+
+ private ContainerLauncher createContainerLauncher(String containerLauncherClassIdentifier,
+ AppContext context,
+ TaskAttemptListener taskAttemptListener,
+ String workingDirectory,
+ Configuration conf) throws
+ UnknownHostException {
+ if (containerLauncherClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+ LOG.info("Creating DefaultContainerLauncher");
+ return new ContainerLauncherImpl(context);
+ } else if (containerLauncherClassIdentifier
+ .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Creating LocalContainerLauncher");
- containerLauncher =
+ return
new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
} else {
- // TODO: Temporary reflection with specific parameters until a clean interface is defined.
- String containerLauncherClassName =
- conf.get(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS);
- if (containerLauncherClassName == null) {
- LOG.info("Creating Default Container Launcher");
- containerLauncher = new ContainerLauncherImpl(context);
- } else {
- LOG.info("Creating container launcher : " + containerLauncherClassName);
- Class<? extends ContainerLauncher> containerLauncherClazz =
- (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
- containerLauncherClassName);
- try {
- Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
- .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
- ctor.setAccessible(true);
- containerLauncher = ctor.newInstance(context, conf, taskAttemptListener);
- } catch (NoSuchMethodException e) {
- throw new TezUncheckedException(e);
- } catch (InvocationTargetException e) {
- throw new TezUncheckedException(e);
- } catch (InstantiationException e) {
- throw new TezUncheckedException(e);
- } catch (IllegalAccessException e) {
- throw new TezUncheckedException(e);
- }
+ LOG.info("Creating container launcher : " + containerLauncherClassIdentifier);
+ Class<? extends ContainerLauncher> containerLauncherClazz =
+ (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
+ containerLauncherClassIdentifier);
+ try {
+ Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
+ .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
+ ctor.setAccessible(true);
+ return ctor.newInstance(context, conf, taskAttemptListener);
+ } catch (NoSuchMethodException e) {
+ throw new TezUncheckedException(e);
+ } catch (InvocationTargetException e) {
+ throw new TezUncheckedException(e);
+ } catch (InstantiationException e) {
+ throw new TezUncheckedException(e);
+ } catch (IllegalAccessException e) {
+ throw new TezUncheckedException(e);
}
-
}
+ // TODO TEZ-2118 Handle routing to multiple launchers
}
@Override
public void serviceInit(Configuration conf) {
- ((AbstractService)containerLauncher).init(conf);
+ for (int i = 0 ; i < containerLaunchers.length ; i++) {
+ ((AbstractService) containerLaunchers[i]).init(conf);
+ }
}
@Override
public void serviceStart() {
- ((AbstractService)containerLauncher).start();
+ for (int i = 0 ; i < containerLaunchers.length ; i++) {
+ ((AbstractService) containerLaunchers[i]).start();
+ }
}
@Override
public void serviceStop() {
- ((AbstractService)containerLauncher).stop();
+ for (int i = 0 ; i < containerLaunchers.length ; i++) {
+ ((AbstractService) containerLaunchers[i]).stop();
+ }
}
@Override
public void handle(NMCommunicatorEvent event) {
- containerLauncher.handle(event);
+ containerLaunchers[0].handle(event);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
index 5c4d43c..c59193c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
@@ -38,11 +38,16 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
private final TaskSpec remoteTaskSpec;
private final TaskAttempt taskAttempt;
+ private final int schedulerId;
+ private final int launcherId;
+ private final int taskCommId;
+
public AMSchedulerEventTALaunchRequest(TezTaskAttemptID attemptId,
Resource capability,
TaskSpec remoteTaskSpec, TaskAttempt ta,
TaskLocationHint locationHint, int priority,
- ContainerContext containerContext) {
+ ContainerContext containerContext,
+ int schedulerId, int launcherId, int taskCommId) {
super(AMSchedulerEventType.S_TA_LAUNCH_REQUEST);
this.attemptId = attemptId;
this.capability = capability;
@@ -51,6 +56,9 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
this.locationHint = locationHint;
this.priority = priority;
this.containerContext = containerContext;
+ this.schedulerId = schedulerId;
+ this.launcherId = launcherId;
+ this.taskCommId = taskCommId;
}
public TezTaskAttemptID getAttemptID() {
@@ -81,6 +89,18 @@ public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
return this.containerContext;
}
+ public int getSchedulerId() {
+ return schedulerId;
+ }
+
+ public int getLauncherId() {
+ return launcherId;
+ }
+
+ public int getTaskCommId() {
+ return taskCommId;
+ }
+
// Parameter replacement: @taskid@ will not be usable
// ProfileTaskRange not available along with ContainerReUse
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index a77b53a..cb109ae 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.TaskLocationHint.TaskBasedLocationAffinity;
@@ -92,7 +93,6 @@ public class TaskSchedulerEventHandler extends AbstractService
@SuppressWarnings("rawtypes")
private final EventHandler eventHandler;
private final String historyUrl;
- protected TaskSchedulerService taskScheduler;
private DAGAppMaster dagAppMaster;
private Map<ApplicationAccessType, String> appAcls = null;
private Thread eventHandlingThread;
@@ -105,14 +105,27 @@ public class TaskSchedulerEventHandler extends AbstractService
private AtomicBoolean shouldUnregisterFlag =
new AtomicBoolean(false);
private final WebUIService webUI;
+ private final String[] taskSchedulerClasses;
+ protected final TaskSchedulerService []taskSchedulers;
BlockingQueue<AMSchedulerEvent> eventQueue
= new LinkedBlockingQueue<AMSchedulerEvent>();
+ /**
+ *
+ * @param appContext
+ * @param clientService
+ * @param eventHandler
+ * @param containerSignatureMatcher
+ * @param webUI
+ * @param schedulerClasses the list of scheduler classes / codes. Tez internal classes are represented as codes.
+ * An empty list defaults to using the YarnTaskScheduler as the only source.
+ */
@SuppressWarnings("rawtypes")
public TaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
- ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
+ ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
+ String [] schedulerClasses) {
super(TaskSchedulerEventHandler.class.getName());
this.appContext = appContext;
this.eventHandler = eventHandler;
@@ -123,6 +136,12 @@ public class TaskSchedulerEventHandler extends AbstractService
if (this.webUI != null) {
this.webUI.setHistoryUrl(this.historyUrl);
}
+ if (schedulerClasses == null || schedulerClasses.length == 0) {
+ this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ } else {
+ this.taskSchedulerClasses = schedulerClasses;
+ }
+ taskSchedulers = new TaskSchedulerService[this.taskSchedulerClasses.length];
}
public Map<ApplicationAccessType, String> getApplicationAcls() {
@@ -139,11 +158,11 @@ public class TaskSchedulerEventHandler extends AbstractService
}
public Resource getAvailableResources() {
- return taskScheduler.getAvailableResources();
+ return taskSchedulers[0].getAvailableResources();
}
public Resource getTotalResources() {
- return taskScheduler.getTotalResources();
+ return taskSchedulers[0].getTotalResources();
}
public synchronized void handleEvent(AMSchedulerEvent sEvent) {
@@ -209,9 +228,9 @@ public class TaskSchedulerEventHandler extends AbstractService
private void handleNodeBlacklistUpdate(AMSchedulerEventNodeBlacklistUpdate event) {
if (event.getType() == AMSchedulerEventType.S_NODE_BLACKLISTED) {
- taskScheduler.blacklistNode(event.getNodeId());
+ taskSchedulers[0].blacklistNode(event.getNodeId());
} else if (event.getType() == AMSchedulerEventType.S_NODE_UNBLACKLISTED) {
- taskScheduler.unblacklistNode(event.getNodeId());
+ taskSchedulers[0].unblacklistNode(event.getNodeId());
} else {
throw new TezUncheckedException("Invalid event type: " + event.getType());
}
@@ -223,14 +242,14 @@ public class TaskSchedulerEventHandler extends AbstractService
// TODO what happens to the task that was connected to this container?
// current assumption is that it will eventually call handleTaStopRequest
//TaskAttempt taskAttempt = (TaskAttempt)
- taskScheduler.deallocateContainer(containerId);
+ taskSchedulers[0].deallocateContainer(containerId);
// TODO does this container need to be stopped via C_STOP_REQUEST
sendEvent(new AMContainerEventStopRequest(containerId));
}
private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) {
TaskAttempt attempt = event.getAttempt();
- boolean wasContainerAllocated = taskScheduler.deallocateTask(attempt, false);
+ boolean wasContainerAllocated = taskSchedulers[0].deallocateTask(attempt, false);
// use stored value of container id in case the scheduler has removed this
// assignment because the task has been deallocated earlier.
// retroactive case
@@ -272,7 +291,7 @@ public class TaskSchedulerEventHandler extends AbstractService
event.getAttemptID()));
}
- boolean wasContainerAllocated = taskScheduler.deallocateTask(attempt, true);
+ boolean wasContainerAllocated = taskSchedulers[0].deallocateTask(attempt, true);
if (!wasContainerAllocated) {
LOG.error("De-allocated successful task: " + attempt.getID()
+ ", but TaskScheduler reported no container assigned to task");
@@ -297,7 +316,7 @@ public class TaskSchedulerEventHandler extends AbstractService
TaskAttempt affinityAttempt = vertex.getTask(taskIndex).getSuccessfulAttempt();
if (affinityAttempt != null) {
Preconditions.checkNotNull(affinityAttempt.getAssignedContainerID(), affinityAttempt.getID());
- taskScheduler.allocateTask(taskAttempt,
+ taskSchedulers[0].allocateTask(taskAttempt,
event.getCapability(),
affinityAttempt.getAssignedContainerID(),
Priority.newInstance(event.getPriority()),
@@ -316,57 +335,59 @@ public class TaskSchedulerEventHandler extends AbstractService
.toArray(new String[locationHint.getRacks().size()]) : null;
}
}
-
- taskScheduler.allocateTask(taskAttempt,
- event.getCapability(),
- hosts,
- racks,
- Priority.newInstance(event.getPriority()),
- event.getContainerContext(),
- event);
- }
-
-
- protected TaskSchedulerService createTaskScheduler(String host, int port,
- String trackingUrl, AppContext appContext) {
- boolean isLocal = getConfig().getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
- TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
- if (isLocal) {
- LOG.info("Using TaskScheduler: LocalTaskSchedulerService");
+
+ taskSchedulers[0].allocateTask(taskAttempt,
+ event.getCapability(),
+ hosts,
+ racks,
+ Priority.newInstance(event.getPriority()),
+ event.getContainerContext(),
+ event);
+ }
+
+ private TaskSchedulerService createTaskScheduler(String host, int port, String trackingUrl,
+ AppContext appContext,
+ String schedulerClassName) {
+ if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+ LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
+ return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
+ host, port, trackingUrl, appContext);
+ } else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+ LOG.info("Creating TaskScheduler: Local TaskScheduler");
return new LocalTaskSchedulerService(this, this.containerSignatureMatcher,
host, port, trackingUrl, appContext);
- }
- else {
- String schedulerClassName = getConfig().get(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS);
- if (schedulerClassName == null) {
- LOG.info("Using TaskScheduler: YarnTaskSchedulerService");
- return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
- host, port, trackingUrl, appContext);
- } else {
- LOG.info("Using custom TaskScheduler: " + schedulerClassName);
- // TODO Temporary reflection with specific parameters. Remove once there is a clean interface.
- Class<? extends TaskSchedulerService> taskSchedulerClazz =
- (Class<? extends TaskSchedulerService>) ReflectionUtils.getClazz(schedulerClassName);
- try {
- Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
- .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
- int.class, String.class, Configuration.class);
- ctor.setAccessible(true);
- TaskSchedulerService taskSchedulerService =
- ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
- return taskSchedulerService;
- } catch (NoSuchMethodException e) {
- throw new TezUncheckedException(e);
- } catch (InvocationTargetException e) {
- throw new TezUncheckedException(e);
- } catch (InstantiationException e) {
- throw new TezUncheckedException(e);
- } catch (IllegalAccessException e) {
- throw new TezUncheckedException(e);
- }
+ } else {
+ LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
+ // TODO TEZ-2003 Temporary reflection with specific parameters. Remove once there is a clean interface.
+ Class<? extends TaskSchedulerService> taskSchedulerClazz =
+ (Class<? extends TaskSchedulerService>) ReflectionUtils.getClazz(schedulerClassName);
+ try {
+ Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
+ .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
+ int.class, String.class, Configuration.class);
+ ctor.setAccessible(true);
+ return ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
+ } catch (NoSuchMethodException e) {
+ throw new TezUncheckedException(e);
+ } catch (InvocationTargetException e) {
+ throw new TezUncheckedException(e);
+ } catch (InstantiationException e) {
+ throw new TezUncheckedException(e);
+ } catch (IllegalAccessException e) {
+ throw new TezUncheckedException(e);
}
}
}
+
+ @VisibleForTesting
+ protected void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
+ // Iterate over the list and create all the taskSchedulers
+ for (int i = 0; i < taskSchedulerClasses.length; i++) {
+ taskSchedulers[i] = createTaskScheduler(host, port,
+ trackingUrl, appContext, taskSchedulerClasses[i]);
+ }
+ }
+
@Override
public synchronized void serviceStart() {
@@ -377,13 +398,17 @@ public class TaskSchedulerEventHandler extends AbstractService
// always try to connect to AM and proxy the response. hence it wont work if the webUIService
// is not enabled.
String trackingUrl = (webUI != null) ? webUI.getTrackingURL() : "";
- taskScheduler = createTaskScheduler(serviceAddr.getHostName(),
- serviceAddr.getPort(), trackingUrl, appContext);
- taskScheduler.init(getConfig());
- taskScheduler.start();
+ instantiateScheduelrs(serviceAddr.getHostName(), serviceAddr.getPort(), trackingUrl, appContext);
+
+ for (int i = 0 ; i < taskSchedulers.length ; i++) {
+ taskSchedulers[i].init(getConfig());
+ taskSchedulers[i].start();
+ }
+
+ // TODO TEZ-2118 Start using multiple task schedulers
if (shouldUnregisterFlag.get()) {
// Flag may have been set earlier when task scheduler was not initialized
- taskScheduler.setShouldUnregister();
+ taskSchedulers[0].setShouldUnregister();
}
this.eventHandlingThread = new Thread("TaskSchedulerEventHandlerThread") {
@@ -432,8 +457,8 @@ public class TaskSchedulerEventHandler extends AbstractService
if (eventHandlingThread != null)
eventHandlingThread.interrupt();
}
- if (taskScheduler != null) {
- ((AbstractService)taskScheduler).stop();
+ if (taskSchedulers[0] != null) {
+ ((AbstractService)taskSchedulers[0]).stop();
}
}
@@ -578,7 +603,7 @@ public class TaskSchedulerEventHandler extends AbstractService
public float getProgress() {
// at this point allocate has been called and so node count must be available
// may change after YARN-1722
- int nodeCount = taskScheduler.getClusterNodeCount();
+ int nodeCount = taskSchedulers[0].getClusterNodeCount();
if (nodeCount != cachedNodeCount) {
cachedNodeCount = nodeCount;
sendEvent(new AMNodeEventNodeCountUpdated(cachedNodeCount));
@@ -593,7 +618,7 @@ public class TaskSchedulerEventHandler extends AbstractService
}
public void dagCompleted() {
- taskScheduler.dagComplete();
+ taskSchedulers[0].dagComplete();
}
public void dagSubmitted() {
@@ -603,7 +628,7 @@ public class TaskSchedulerEventHandler extends AbstractService
@Override
public void preemptContainer(ContainerId containerId) {
- taskScheduler.deallocateContainer(containerId);
+ taskSchedulers[0].deallocateContainer(containerId);
// Inform the Containers about completion.
sendEvent(new AMContainerEventCompleted(containerId, ContainerExitStatus.INVALID,
"Container preempted internally", TaskAttemptTerminationCause.INTERNAL_PREEMPTION));
@@ -612,13 +637,13 @@ public class TaskSchedulerEventHandler extends AbstractService
public void setShouldUnregisterFlag() {
LOG.info("TaskScheduler notified that it should unregister from RM");
this.shouldUnregisterFlag.set(true);
- if (this.taskScheduler != null) {
- this.taskScheduler.setShouldUnregister();
+ if (this.taskSchedulers[0] != null) {
+ this.taskSchedulers[0].setShouldUnregister();
}
}
public boolean hasUnregistered() {
- return this.taskScheduler.hasUnregistered();
+ return this.taskSchedulers[0].hasUnregistered();
}
@VisibleForTesting
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 999fe1c..8763a0c 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -512,10 +512,11 @@ public class MockDAGAppMaster extends DAGAppMaster {
this.handlerConcurrency = handlerConcurrency;
this.numConcurrentContainers = numConcurrentContainers;
}
-
+
// use mock container launcher for tests
@Override
- protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf)
+ protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
+ String[] containerLaunchers)
throws UnknownHostException {
return new ContainerLauncherRouter(containerLauncher);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index c454c7c..219217f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -35,6 +35,7 @@ import java.util.Map;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -123,12 +124,12 @@ public class TestTaskAttemptListenerImplTezDag {
doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
doReturn(container).when(amContainer).getContainer();
- taskAttemptListener =
- new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
- mock(ContainerHeartbeatHandler.class), null);
+ taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
taskSpec = mock(TaskSpec.class);
doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
amContainerTask = new AMContainerTask(taskSpec, null, null, false, 0);
@@ -138,6 +139,9 @@ public class TestTaskAttemptListenerImplTezDag {
@Test(timeout = 5000)
public void testGetTask() throws IOException {
+ TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+ TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
ContainerId containerId1 = createContainerId(appId, 1);
ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
containerTask = tezUmbilical.getTask(containerContext1);
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 79450a9..6ea1388 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -1399,7 +1399,7 @@ public class TestContainerReuse {
InputDescriptor.create("inputClassName"), 1)),
Collections.singletonList(new OutputSpec("vertexName",
OutputDescriptor.create("outputClassName"), 1)), null), ta, locationHint,
- priority.getPriority(), containerContext);
+ priority.getPriority(), containerContext, 0, 0, 0);
return lr;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index fc7aa50..b7a3a87 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -89,13 +89,13 @@ public class TestTaskSchedulerEventHandler {
public MockTaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
- super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI);
+ super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new String[] {});
}
-
+
@Override
- protected TaskSchedulerService createTaskScheduler(String host, int port,
- String trackingUrl, AppContext appContext) {
- return mockTaskScheduler;
+ protected void instantiateScheduelrs(String host, int port, String trackingUrl,
+ AppContext appContext) {
+ taskSchedulers[0] = mockTaskScheduler;
}
@Override
@@ -194,7 +194,7 @@ public class TestTaskSchedulerEventHandler {
when(mockAppContext.getCurrentDAG().getVertex(affVertexName)).thenReturn(affVertex);
Resource resource = Resource.newInstance(100, 1);
AMSchedulerEventTALaunchRequest event = new AMSchedulerEventTALaunchRequest
- (taId, resource, null, mockTaskAttempt, locHint, 3, null);
+ (taId, resource, null, mockTaskAttempt, locHint, 3, null, 0, 0, 0);
schedulerHandler.notify.set(false);
schedulerHandler.handle(event);
synchronized (schedulerHandler.notify) {
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index 77c98b7..d775300 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -127,31 +127,29 @@ class TestTaskSchedulerHelpers {
EventHandler eventHandler,
TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
ContainerSignatureMatcher containerSignatureMatcher) {
- super(appContext, null, eventHandler, containerSignatureMatcher, null);
+ super(appContext, null, eventHandler, containerSignatureMatcher, null, new String[]{});
this.amrmClientAsync = amrmClientAsync;
this.containerSignatureMatcher = containerSignatureMatcher;
}
@Override
- public TaskSchedulerService createTaskScheduler(String host, int port,
- String trackingUrl, AppContext appContext) {
- return new TaskSchedulerWithDrainableAppCallback(this,
+ public void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
+ taskSchedulers[0] = new TaskSchedulerWithDrainableAppCallback(this,
containerSignatureMatcher, host, port, trackingUrl, amrmClientAsync,
appContext);
}
public TaskSchedulerService getSpyTaskScheduler() {
- return this.taskScheduler;
+ return taskSchedulers[0];
}
@Override
public void serviceStart() {
- TaskSchedulerService taskSchedulerReal = createTaskScheduler("host", 0, "",
- appContext);
+ instantiateScheduelrs("host", 0, "", appContext);
// Init the service so that reuse configuration is picked up.
- ((AbstractService)taskSchedulerReal).init(getConfig());
- ((AbstractService)taskSchedulerReal).start();
- taskScheduler = spy(taskSchedulerReal);
+ ((AbstractService)taskSchedulers[0]).init(getConfig());
+ ((AbstractService)taskSchedulers[0]).start();
+ taskSchedulers[0] = spy(taskSchedulers[0]);
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/b0568218/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index a93c1a4..ae7e7f8 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -45,6 +45,8 @@ public class TestExternalTezServices {
private static final Log LOG = LogFactory.getLog(TestExternalTezServices.class);
+ private static final String EXT_PUSH_ENTITY_NAME = "ExtServiceTestPush";
+
private static MiniTezCluster tezCluster;
private static MiniDFSCluster dfsCluster;
private static MiniTezTestServiceCluster tezTestServiceCluster;
@@ -106,12 +108,17 @@ public class TestExternalTezServices {
remoteFs.mkdirs(stagingDirPath);
// This is currently configured to push tasks into the Service, and then use the standard RPC
confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
- confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS,
- TezTestServiceTaskSchedulerService.class.getName());
- confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS,
- TezTestServiceNoOpContainerLauncher.class.getName());
- confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS,
- TezTestServiceTaskCommunicatorImpl.class.getName());
+ confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
+ EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskSchedulerService.class.getName());
+ confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
+ EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
+ confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
+ EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
+
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
+
TezConfiguration tezConf = new TezConfiguration(confForJobs);
[06/51] [abbrv] tez git commit: TEZ-2123. Fix component managers to
use pluggable components. Enable hybrid mode. (sseth)
Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 6ea1388..656bca1 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -223,7 +223,7 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(
- ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED));
+ ta11, containerHost1.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
verify(taskSchedulerEventHandler, times(1)).taskAllocated(
@@ -235,7 +235,7 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
- TaskAttemptState.SUCCEEDED));
+ TaskAttemptState.SUCCEEDED, 0));
long currentTs = System.currentTimeMillis();
Throwable exception = null;
@@ -356,7 +356,7 @@ public class TestContainerReuse {
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta21, containerHost2.getId(),
- TaskAttemptState.SUCCEEDED));
+ TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta21), eq(true));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(
@@ -459,7 +459,7 @@ public class TestContainerReuse {
verify(taskSchedulerEventHandler).taskAllocated(eq(ta11), any(Object.class), eq(container1));
// Task assigned to container completed successfully. Container should be re-used.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta12), any(Object.class), eq(container1));
@@ -469,7 +469,7 @@ public class TestContainerReuse {
// Task assigned to container completed successfully.
// Verify reuse across hosts.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta12, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta12), eq(true));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta13), any(Object.class), eq(container1));
@@ -478,7 +478,7 @@ public class TestContainerReuse {
eventHandler.reset();
// Verify no re-use if a previous task fails.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, 0));
drainableAppCallback.drain();
verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class), eq(container1));
verify(taskScheduler).deallocateTask(eq(ta13), eq(false));
@@ -496,7 +496,7 @@ public class TestContainerReuse {
verify(taskSchedulerEventHandler).taskAllocated(eq(ta14), any(Object.class), eq(container2));
// Task assigned to container completed successfully. No pending requests. Container should be released.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta14, container2.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta14), eq(true));
verify(rmClient).releaseAssignedContainer(eq(container2.getId()));
@@ -607,7 +607,7 @@ public class TestContainerReuse {
// First task had profiling on. This container can not be reused further.
taskSchedulerEventHandler.handleEvent(
- new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED));
+ new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta12), any(Object.class),
@@ -653,7 +653,7 @@ public class TestContainerReuse {
// Verify that the container can not be reused when profiling option is turned on
// Even for 2 tasks having same profiling option can have container reusability.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta13, container2.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta13), eq(true));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(eq(ta14), any(Object.class),
@@ -698,7 +698,7 @@ public class TestContainerReuse {
verify(taskSchedulerEventHandler).taskAllocated(eq(ta15), any(Object.class), eq(container3));
//Ensure task 6 (of vertex 1) is allocated to same container
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta15, container3.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta15), eq(true));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta16), any(Object.class), eq(container3));
@@ -811,7 +811,7 @@ public class TestContainerReuse {
// until delay expires.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta11, container1.getId(),
- TaskAttemptState.SUCCEEDED));
+ TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
verify(taskSchedulerEventHandler, times(0)).taskAllocated(
@@ -828,7 +828,7 @@ public class TestContainerReuse {
// TA12 completed.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta12, container1.getId(),
- TaskAttemptState.SUCCEEDED));
+ TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
LOG.info("Sleeping to ensure that the scheduling loop runs");
Thread.sleep(3000l);
@@ -946,7 +946,7 @@ public class TestContainerReuse {
// Container should be assigned to task21.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta11, container1.getId(),
- TaskAttemptState.SUCCEEDED));
+ TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta11), eq(true));
verify(taskSchedulerEventHandler).taskAllocated(
@@ -956,7 +956,7 @@ public class TestContainerReuse {
// Task 2 completes.
taskSchedulerEventHandler.handleEvent(
new AMSchedulerEventTAEnded(ta21, container1.getId(),
- TaskAttemptState.SUCCEEDED));
+ TaskAttemptState.SUCCEEDED, 0));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
LOG.info("Sleeping to ensure that the scheduling loop runs");
@@ -1065,7 +1065,7 @@ public class TestContainerReuse {
assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
// Task assigned to container completed successfully. Container should be re-used.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta111), eq(true));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
@@ -1077,7 +1077,7 @@ public class TestContainerReuse {
// Task assigned to container completed successfully.
// Verify reuse across hosts.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta112), eq(true));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
@@ -1118,7 +1118,7 @@ public class TestContainerReuse {
assertEquals(2, assignEvent.getRemoteTaskLocalResources().size());
eventHandler.reset();
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta211, container1.getId(), TaskAttemptState.SUCCEEDED, 0));
drainableAppCallback.drain();
verify(taskScheduler).deallocateTask(eq(ta211), eq(true));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta212), any(Object.class), eq(container1));
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
index 60782e6..12390b2 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskScheduler.java
@@ -59,7 +59,7 @@ public class TestLocalTaskScheduler {
TezConfiguration tezConf = new TezConfiguration();
tezConf.setInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS, MAX_TASKS);
- LocalContainerFactory containerFactory = new LocalContainerFactory(createMockAppContext());
+ LocalContainerFactory containerFactory = new LocalContainerFactory(createMockAppContext(), 1000);
HashMap<Object, Container> taskAllocations = new LinkedHashMap<Object, Container>();
PriorityBlockingQueue<TaskRequest> taskRequestQueue = new PriorityBlockingQueue<TaskRequest>();
TaskSchedulerAppCallback appClientDelegate = mock(TaskSchedulerAppCallback.class);
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
index 3cf4f6c..25cf4b5 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
@@ -22,6 +22,8 @@ import java.util.HashMap;
import java.util.concurrent.BlockingQueue;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
@@ -81,8 +83,12 @@ public class TestLocalTaskSchedulerService {
*/
@Test(timeout = 5000)
public void testDeallocationBeforeAllocation() {
+ AppContext appContext = mock(AppContext.class);
+ ApplicationAttemptId appAttemptId =
+ ApplicationAttemptId.newInstance(ApplicationId.newInstance(10000l, 1), 1);
+ doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce
- (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", mock(AppContext.class));
+ (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", appContext);
taskSchedulerService.init(new Configuration());
taskSchedulerService.start();
@@ -105,8 +111,12 @@ public class TestLocalTaskSchedulerService {
*/
@Test(timeout = 5000)
public void testDeallocationAfterAllocation() {
+ AppContext appContext = mock(AppContext.class);
+ ApplicationAttemptId appAttemptId =
+ ApplicationAttemptId.newInstance(ApplicationId.newInstance(10000l, 1), 1);
+ doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
MockLocalTaskSchedulerSerivce taskSchedulerService = new MockLocalTaskSchedulerSerivce
- (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", mock(AppContext.class));
+ (mock(TaskSchedulerAppCallback.class), mock(ContainerSignatureMatcher.class), "", 0, "", appContext);
taskSchedulerService.init(new Configuration());
taskSchedulerService.start();
@@ -132,13 +142,13 @@ public class TestLocalTaskSchedulerService {
String appHostName, int appHostPort, String appTrackingUrl,
AppContext appContext) {
super(appClient, containerSignatureMatcher, appHostName, appHostPort,
- appTrackingUrl, appContext);
+ appTrackingUrl, 10000l, appContext);
}
@Override
public AsyncDelegateRequestHandler createRequestHandler(Configuration conf) {
requestHandler = new MockAsyncDelegateRequestHandler(taskRequestQueue,
- new LocalContainerFactory(appContext),
+ new LocalContainerFactory(appContext, customContainerAppId),
taskAllocations,
appClientDelegate,
conf);
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index b7a3a87..daf1db6 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -89,7 +89,7 @@ public class TestTaskSchedulerEventHandler {
public MockTaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
- super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new String[] {});
+ super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new String[] {}, false);
}
@Override
@@ -162,7 +162,7 @@ public class TestTaskSchedulerEventHandler {
AMSchedulerEventTALaunchRequest lr =
new AMSchedulerEventTALaunchRequest(mockAttemptId, resource, null, mockTaskAttempt, locHint,
- priority, containerContext);
+ priority, containerContext, 0, 0, 0);
schedulerHandler.taskAllocated(mockTaskAttempt, lr, container);
assertEquals(2, mockEventHandler.events.size());
assertTrue(mockEventHandler.events.get(1) instanceof AMContainerEventAssignTA);
@@ -249,9 +249,14 @@ public class TestTaskSchedulerEventHandler {
Configuration conf = new Configuration(false);
schedulerHandler.init(conf);
schedulerHandler.start();
-
+
+ AMContainer mockAmContainer = mock(AMContainer.class);
+ when(mockAmContainer.getTaskSchedulerIdentifier()).thenReturn(0);
+ when(mockAmContainer.getContainerLauncherIdentifier()).thenReturn(0);
+ when(mockAmContainer.getTaskCommunicatorIdentifier()).thenReturn(0);
ContainerId mockCId = mock(ContainerId.class);
verify(mockTaskScheduler, times(0)).deallocateContainer((ContainerId)any());
+ when(mockAMContainerMap.get(mockCId)).thenReturn(mockAmContainer);
schedulerHandler.preemptContainer(mockCId);
verify(mockTaskScheduler, times(1)).deallocateContainer(mockCId);
assertEquals(1, mockEventHandler.events.size());
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index d775300..ffab769 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -127,7 +127,7 @@ class TestTaskSchedulerHelpers {
EventHandler eventHandler,
TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
ContainerSignatureMatcher containerSignatureMatcher) {
- super(appContext, null, eventHandler, containerSignatureMatcher, null, new String[]{});
+ super(appContext, null, eventHandler, containerSignatureMatcher, null, new String[]{}, false);
this.amrmClientAsync = amrmClientAsync;
this.containerSignatureMatcher = containerSignatureMatcher;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index fafbba6..bdd0f61 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerHeartbeatHandler;
import org.apache.tez.dag.app.ContainerContext;
@@ -104,7 +105,7 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.LAUNCHING);
// 1 Launch request.
wc.verifyCountAndGetOutgoingEvents(1);
- verify(wc.tal).registerRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
assertNull(wc.amContainer.getCurrentTaskAttempt());
// Assign task.
@@ -121,7 +122,7 @@ public class TestAMContainer {
// Once for the previous NO_TASKS, one for the actual task.
verify(wc.chh).register(wc.containerID);
ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
assertEquals(1, argumentCaptor.getAllValues().size());
assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
assertEquals(WrappedContainer.taskPriority, argumentCaptor.getAllValues().get(0).getPriority());
@@ -131,14 +132,14 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
assertNull(wc.amContainer.getCurrentTaskAttempt());
- verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
+ verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
// Container completed
wc.containerCompleted();
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -157,7 +158,7 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.LAUNCHING);
// 1 Launch request.
wc.verifyCountAndGetOutgoingEvents(1);
- verify(wc.tal).registerRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
// Container Launched
wc.containerLaunched();
@@ -172,7 +173,7 @@ public class TestAMContainer {
wc.verifyNoOutgoingEvents();
assertEquals(wc.taskAttemptID, wc.amContainer.getCurrentTaskAttempt());
ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
assertEquals(1, argumentCaptor.getAllValues().size());
assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
@@ -180,13 +181,13 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
assertNull(wc.amContainer.getCurrentTaskAttempt());
- verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
+ verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
wc.containerCompleted();
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
@@ -205,7 +206,7 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.LAUNCHING);
// 1 Launch request.
wc.verifyCountAndGetOutgoingEvents(1);
- verify(wc.tal).registerRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
assertNull(wc.amContainer.getCurrentTaskAttempt());
// Assign task.
@@ -222,7 +223,7 @@ public class TestAMContainer {
// Once for the previous NO_TASKS, one for the actual task.
verify(wc.chh).register(wc.containerID);
ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
assertEquals(1, argumentCaptor.getAllValues().size());
assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
@@ -231,13 +232,13 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
assertNull(wc.amContainer.getCurrentTaskAttempt());
- verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
+ verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID, 0);
TezTaskAttemptID taId2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taId2);
wc.verifyState(AMContainerState.RUNNING);
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
assertEquals(2, argumentCaptor.getAllValues().size());
assertEquals(taId2, argumentCaptor.getAllValues().get(1).getTask().getTaskAttemptID());
@@ -246,14 +247,14 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
assertNull(wc.amContainer.getCurrentTaskAttempt());
- verify(wc.tal).unregisterTaskAttempt(taId2);
+ verify(wc.tal).unregisterTaskAttempt(taId2, 0);
// Container completed
wc.containerCompleted();
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
@@ -286,7 +287,7 @@ public class TestAMContainer {
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -323,7 +324,7 @@ public class TestAMContainer {
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
wc.verifyNoOutgoingEvents();
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
assertNull(wc.amContainer.getCurrentTaskAttempt());
@@ -346,7 +347,7 @@ public class TestAMContainer {
wc.assignTaskAttempt(taID2);
wc.verifyState(AMContainerState.STOP_REQUESTED);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
// 1 for NM stop request. 2 TERMINATING to TaskAttempt.
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -378,13 +379,13 @@ public class TestAMContainer {
wc.launchContainer();
wc.assignTaskAttempt(wc.taskAttemptID);
wc.verifyState(AMContainerState.LAUNCHING);
- verify(wc.tal).registerRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2);
wc.verifyState(AMContainerState.STOP_REQUESTED);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
// 1 for NM stop request. 2 TERMINATING to TaskAttempt.
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
@@ -420,7 +421,7 @@ public class TestAMContainer {
wc.containerTimedOut();
wc.verifyState(AMContainerState.STOP_REQUESTED);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
// 1 to TA, 1 for RM de-allocate.
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -454,7 +455,7 @@ public class TestAMContainer {
wc.stopRequest();
wc.verifyState(AMContainerState.STOP_REQUESTED);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).unregister(wc.containerID);
// 1 to TA, 1 for RM de-allocate.
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
@@ -484,11 +485,11 @@ public class TestAMContainer {
wc.launchContainer();
wc.assignTaskAttempt(wc.taskAttemptID);
wc.verifyState(AMContainerState.LAUNCHING);
- verify(wc.tal).registerRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
wc.launchFailed();
wc.verifyState(AMContainerState.STOPPING);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -537,8 +538,8 @@ public class TestAMContainer {
wc.containerCompleted();
wc.verifyState(AMContainerState.COMPLETED);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -567,8 +568,8 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
wc.verifyState(AMContainerState.COMPLETED);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -598,8 +599,8 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.ABORTED, TaskAttemptTerminationCause.NODE_FAILED);
wc.verifyState(AMContainerState.COMPLETED);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
@@ -629,8 +630,8 @@ public class TestAMContainer {
wc.containerCompleted();
wc.verifyState(AMContainerState.COMPLETED);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -658,8 +659,8 @@ public class TestAMContainer {
wc.containerCompleted();
wc.verifyState(AMContainerState.COMPLETED);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -693,8 +694,8 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.PREEMPTED, TaskAttemptTerminationCause.EXTERNAL_PREEMPTION);
wc.verifyState(AMContainerState.COMPLETED);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -730,8 +731,8 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.INVALID, TaskAttemptTerminationCause.INTERNAL_PREEMPTION);
wc.verifyState(AMContainerState.COMPLETED);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -767,8 +768,8 @@ public class TestAMContainer {
wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
wc.verifyState(AMContainerState.COMPLETED);
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.tal).registerRunningContainer(wc.containerID, 0);
+ verify(wc.tal).unregisterRunningContainer(wc.containerID, 0);
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -1011,7 +1012,7 @@ public class TestAMContainer {
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
AMContainerTask task1 = argumentCaptor.getAllValues().get(0);
assertEquals(0, task1.getAdditionalResources().size());
wc.taskAttemptSucceeded(wc.taskAttemptID);
@@ -1024,7 +1025,7 @@ public class TestAMContainer {
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2, additionalResources, new Credentials());
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
AMContainerTask task2 = argumentCaptor.getAllValues().get(1);
Map<String, LocalResource> pullTaskAdditionalResources = task2.getAdditionalResources();
assertEquals(2, pullTaskAdditionalResources.size());
@@ -1047,7 +1048,7 @@ public class TestAMContainer {
TezTaskAttemptID taID3 = TezTaskAttemptID.getInstance(wc.taskID, 3);
wc.assignTaskAttempt(taID3, new HashMap<String, LocalResource>(), new Credentials());
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
AMContainerTask task3 = argumentCaptor.getAllValues().get(2);
assertEquals(0, task3.getAdditionalResources().size());
wc.taskAttemptSucceeded(taID3);
@@ -1100,7 +1101,7 @@ public class TestAMContainer {
wc.containerLaunched();
wc.assignTaskAttempt(attempt11, LRs, dag1Credentials);
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
fetchedTask = argumentCaptor.getAllValues().get(0);
assertTrue(fetchedTask.haveCredentialsChanged());
assertNotNull(fetchedTask.getCredentials());
@@ -1109,7 +1110,7 @@ public class TestAMContainer {
wc.assignTaskAttempt(attempt12, LRs, dag1Credentials);
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
fetchedTask = argumentCaptor.getAllValues().get(1);
assertFalse(fetchedTask.haveCredentialsChanged());
assertNull(fetchedTask.getCredentials());
@@ -1119,7 +1120,7 @@ public class TestAMContainer {
wc.setNewDAGID(dagID2);
wc.assignTaskAttempt(attempt21, LRs, null);
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
fetchedTask = argumentCaptor.getAllValues().get(2);
assertTrue(fetchedTask.haveCredentialsChanged());
assertNull(fetchedTask.getCredentials());
@@ -1127,7 +1128,7 @@ public class TestAMContainer {
wc.assignTaskAttempt(attempt22, LRs, null);
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(4)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(4)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
fetchedTask = argumentCaptor.getAllValues().get(3);
assertFalse(fetchedTask.haveCredentialsChanged());
assertNull(fetchedTask.getCredentials());
@@ -1137,7 +1138,7 @@ public class TestAMContainer {
wc.setNewDAGID(dagID3);
wc.assignTaskAttempt(attempt31, LRs , dag3Credentials);
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(5)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(5)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
fetchedTask = argumentCaptor.getAllValues().get(4);
assertTrue(fetchedTask.haveCredentialsChanged());
assertNotNull(fetchedTask.getCredentials());
@@ -1147,7 +1148,7 @@ public class TestAMContainer {
wc.assignTaskAttempt(attempt32, LRs, dag1Credentials);
argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
- verify(wc.tal, times(6)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ verify(wc.tal, times(6)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID), eq(0));
fetchedTask = argumentCaptor.getAllValues().get(5);
assertFalse(fetchedTask.haveCredentialsChanged());
assertNull(fetchedTask.getCredentials());
@@ -1200,9 +1201,10 @@ public class TestAMContainer {
chh = mock(ContainerHeartbeatHandler.class);
- InetSocketAddress addr = new InetSocketAddress("localhost", 0);
tal = mock(TaskAttemptListener.class);
- doReturn(addr).when(tal).getAddress();
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(tal).getTaskCommunicator(0);
dagID = TezDAGID.getInstance(applicationID, 1);
vertexID = TezVertexID.getInstance(dagID, 1);
@@ -1228,7 +1230,7 @@ public class TestAMContainer {
doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
amContainer = new AMContainerImpl(container, chh, tal,
- new ContainerContextMatcher(), appContext);
+ new ContainerContextMatcher(), appContext, 0, 0, 0);
}
public WrappedContainer() {
@@ -1278,7 +1280,7 @@ public class TestAMContainer {
Token<JobTokenIdentifier> jobToken = mock(Token.class);
TokenCache.setSessionToken(jobToken, credentials);
amContainer.handle(new AMContainerEventLaunchRequest(containerID, vertexID,
- new ContainerContext(localResources, credentials, new HashMap<String, String>(), "")));
+ new ContainerContext(localResources, credentials, new HashMap<String, String>(), ""), 0, 0));
}
public void assignTaskAttempt(TezTaskAttemptID taID) {
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
index 61371e8..dee4541 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerHeartbeatHandler;
import org.apache.tez.dag.app.TaskAttemptListener;
@@ -43,8 +44,9 @@ public class TestAMContainerMap {
private TaskAttemptListener mockTaskAttemptListener() {
TaskAttemptListener tal = mock(TaskAttemptListener.class);
- InetSocketAddress socketAddr = new InetSocketAddress("localhost", 21000);
- doReturn(socketAddr).when(tal).getAddress();
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 21000)).when(taskComm).getAddress();
+ doReturn(taskComm).when(tal).getTaskCommunicator(0);
return tal;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
index 1d124a6..ba17ba0 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
@@ -20,6 +20,7 @@ package org.apache.tez.examples;
import java.io.IOException;
import java.util.Set;
+import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -135,7 +136,7 @@ public class JoinValidate extends TezExampleBase {
private DAG createDag(TezConfiguration tezConf, Path lhs, Path rhs, int numPartitions)
throws IOException {
- DAG dag = DAG.create("JoinValidate");
+ DAG dag = DAG.create(getDagName());
// Configuration for intermediate output - shared by Vertex1 and Vertex2
// This should only be setting selective keys from the underlying conf. Fix after there's a
@@ -152,15 +153,18 @@ public class JoinValidate extends TezExampleBase {
MRInput
.createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
lhs.toUri().toString()).groupSplits(!isDisableSplitGrouping()).build());
+ setVertexProperties(lhsVertex, getLhsVertexProperties());
Vertex rhsVertex = Vertex.create(RHS_INPUT_NAME, ProcessorDescriptor.create(
ForwardingProcessor.class.getName())).addDataSource("rhs",
MRInput
.createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
rhs.toUri().toString()).groupSplits(!isDisableSplitGrouping()).build());
+ setVertexProperties(rhsVertex, getRhsVertexProperties());
Vertex joinValidateVertex = Vertex.create("joinvalidate", ProcessorDescriptor.create(
JoinValidateProcessor.class.getName()), numPartitions);
+ setVertexProperties(joinValidateVertex, getValidateVertexProperties());
Edge e1 = Edge.create(lhsVertex, joinValidateVertex, edgeConf.createDefaultEdgeProperty());
Edge e2 = Edge.create(rhsVertex, joinValidateVertex, edgeConf.createDefaultEdgeProperty());
@@ -170,6 +174,30 @@ public class JoinValidate extends TezExampleBase {
return dag;
}
+ private void setVertexProperties(Vertex vertex, Map<String, String> properties) {
+ if (properties != null) {
+ for (Map.Entry<String, String> entry : properties.entrySet()) {
+ vertex.setConf(entry.getKey(), entry.getValue());
+ }
+ }
+ }
+
+ protected Map<String, String> getLhsVertexProperties() {
+ return null;
+ }
+
+ protected Map<String, String> getRhsVertexProperties() {
+ return null;
+ }
+
+ protected Map<String, String> getValidateVertexProperties() {
+ return null;
+ }
+
+ protected String getDagName() {
+ return "JoinValidate";
+ }
+
public static class JoinValidateProcessor extends SimpleProcessor {
private static final Logger LOG = LoggerFactory.getLogger(JoinValidateProcessor.class);
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index e83165b..27356bc 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -14,6 +14,8 @@
package org.apache.tez.dag.app.launcher;
+import java.net.InetSocketAddress;
+
import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
import org.apache.commons.logging.Log;
@@ -124,7 +126,8 @@ public class TezTestServiceContainerLauncher extends AbstractService implements
private RunContainerRequestProto constructRunContainerRequest(NMCommunicatorLaunchRequestEvent event) {
RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
- builder.setAmHost(tal.getAddress().getHostName()).setAmPort(tal.getAddress().getPort());
+ InetSocketAddress address = tal.getTaskCommunicator(event.getTaskCommId()).getAddress();
+ builder.setAmHost(address.getHostName()).setAmPort(address.getPort());
builder.setAppAttemptNumber(event.getContainer().getId().getApplicationAttemptId().getAttemptId());
builder.setApplicationIdString(
event.getContainer().getId().getApplicationAttemptId().getApplicationId().toString());
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index e3c18bf..5657f86 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -14,7 +14,6 @@
package org.apache.tez.dag.app.rm;
-import java.io.IOException;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
@@ -32,25 +31,17 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.client.api.AMRMClient;
-import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.service.TezTestServiceConfConstants;
-// TODO Registration with RM - so that the AM is considered dead and restarted in the expiry interval - 10 minutes.
-
public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
private static final Log LOG = LogFactory.getLog(TezTestServiceTaskSchedulerService.class);
@@ -71,7 +62,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
private final ConcurrentMap<Object, ContainerId> runningTasks =
new ConcurrentHashMap<Object, ContainerId>();
- private final AMRMClientAsync<AMRMClient.ContainerRequest> amRmClient;
+ // AppIdIdentifier to avoid conflicts with other containres in the system.
// Per instance
private final int memoryPerInstance;
@@ -82,10 +73,13 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
private final Resource resourcePerContainer;
+ // Not registering with the RM. Assuming the main TezScheduler will always run (except local mode),
+ // and take care of YARN registration.
public TezTestServiceTaskSchedulerService(TaskSchedulerAppCallback appClient,
AppContext appContext,
String clientHostname, int clientPort,
String trackingUrl,
+ long customAppIdIdentifier,
Configuration conf) {
// Accepting configuration here to allow setting up fields as final
super(TezTestServiceTaskSchedulerService.class.getName());
@@ -93,7 +87,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
this.appClientDelegate = createAppCallbackDelegate(appClient);
this.appContext = appContext;
this.serviceHosts = new LinkedList<String>();
- this.containerFactory = new ContainerFactory(appContext);
+ this.containerFactory = new ContainerFactory(appContext, customAppIdIdentifier);
this.memoryPerInstance = conf
.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, -1);
@@ -123,7 +117,6 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
int memoryPerContainer = (int) (memoryPerInstance / (float) executorsPerInstance);
int coresPerContainer = (int) (coresPerInstance / (float) executorsPerInstance);
this.resourcePerContainer = Resource.newInstance(memoryPerContainer, coresPerContainer);
- this.amRmClient = TezAMRMClientAsync.createAMRMClientAsync(5000, new FakeAmRmCallbackHandler());
String[] hosts = conf.getTrimmedStrings(TezTestServiceConfConstants.TEZ_TEST_SERVICE_HOSTS);
if (hosts == null || hosts.length == 0) {
@@ -143,36 +136,8 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
}
@Override
- public void serviceInit(Configuration conf) {
- amRmClient.init(conf);
- }
-
- @Override
- public void serviceStart() {
- amRmClient.start();
- RegisterApplicationMasterResponse response;
- try {
- amRmClient.registerApplicationMaster(clientHostname, clientPort, trackingUrl);
- } catch (YarnException e) {
- throw new TezUncheckedException(e);
- } catch (IOException e) {
- throw new TezUncheckedException(e);
- }
- }
-
- @Override
public void serviceStop() {
if (!this.isStopped.getAndSet(true)) {
-
- try {
- TaskSchedulerAppCallback.AppFinalStatus status = appClientDelegate.getFinalAppStatus();
- amRmClient.unregisterApplicationMaster(status.exitStatus, status.exitMessage,
- status.postCompletionTrackingUrl);
- } catch (YarnException e) {
- throw new TezUncheckedException(e);
- } catch (IOException e) {
- throw new TezUncheckedException(e);
- }
appCallbackExecutor.shutdownNow();
}
}
@@ -264,7 +229,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
private ExecutorService createAppCallbackExecutorService() {
return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
- .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
+ .setNameFormat("TezTestTaskSchedulerAppCaller").setDaemon(true).build());
}
private TaskSchedulerAppCallback createAppCallbackDelegate(
@@ -274,7 +239,7 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
}
private String selectHost(String[] requestedHosts) {
- String host = null;
+ String host;
if (requestedHosts != null && requestedHosts.length > 0) {
Arrays.sort(requestedHosts);
host = requestedHosts[0];
@@ -287,17 +252,19 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
}
static class ContainerFactory {
- final AppContext appContext;
AtomicInteger nextId;
-
- public ContainerFactory(AppContext appContext) {
- this.appContext = appContext;
- this.nextId = new AtomicInteger(2);
+ final ApplicationAttemptId customAppAttemptId;
+
+ public ContainerFactory(AppContext appContext, long appIdLong) {
+ this.nextId = new AtomicInteger(1);
+ ApplicationId appId = ApplicationId
+ .newInstance(appIdLong, appContext.getApplicationAttemptId().getApplicationId().getId());
+ this.customAppAttemptId = ApplicationAttemptId
+ .newInstance(appId, appContext.getApplicationAttemptId().getAttemptId());
}
public Container createContainer(Resource capability, Priority priority, String hostname, int port) {
- ApplicationAttemptId appAttemptId = appContext.getApplicationAttemptId();
- ContainerId containerId = ContainerId.newInstance(appAttemptId, nextId.getAndIncrement());
+ ContainerId containerId = ContainerId.newInstance(customAppAttemptId, nextId.getAndIncrement());
NodeId nodeId = NodeId.newInstance(hostname, port);
String nodeHttpAddress = "hostname:0";
@@ -311,37 +278,4 @@ public class TezTestServiceTaskSchedulerService extends TaskSchedulerService {
return container;
}
}
-
- private static class FakeAmRmCallbackHandler implements AMRMClientAsync.CallbackHandler {
-
- @Override
- public void onContainersCompleted(List<ContainerStatus> statuses) {
-
- }
-
- @Override
- public void onContainersAllocated(List<Container> containers) {
-
- }
-
- @Override
- public void onShutdownRequest() {
-
- }
-
- @Override
- public void onNodesUpdated(List<NodeReport> updatedNodes) {
-
- }
-
- @Override
- public float getProgress() {
- return 0;
- }
-
- @Override
- public void onError(Throwable e) {
-
- }
- }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java b/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
new file mode 100644
index 0000000..e5d2e3b
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
@@ -0,0 +1,53 @@
+/*
+ * 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.tez.examples;
+
+import java.util.Map;
+
+public class JoinValidateConfigured extends JoinValidate {
+
+ private final Map<String, String> lhsProps;
+ private final Map<String, String> rhsProps;
+ private final Map<String, String> validateProps;
+ private final String dagNameSuffix;
+
+ public JoinValidateConfigured(Map<String, String> lhsProps, Map<String, String> rhsProps,
+ Map<String, String> validateProps, String dagNameSuffix) {
+ this.lhsProps = lhsProps;
+ this.rhsProps = rhsProps;
+ this.validateProps = validateProps;
+ this.dagNameSuffix = dagNameSuffix;
+ }
+
+ @Override
+ protected Map<String, String> getLhsVertexProperties() {
+ return this.lhsProps;
+ }
+
+ @Override
+ protected Map<String, String> getRhsVertexProperties() {
+ return this.rhsProps;
+ }
+
+ @Override
+ protected Map<String, String> getValidateVertexProperties() {
+ return this.validateProps;
+ }
+
+ @Override
+ protected String getDagName() {
+ return "JoinValidate_" + dagNameSuffix;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index ae7e7f8..9c149c6 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -15,11 +15,11 @@
package org.apache.tez.tests;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Map;
+import com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -28,13 +28,14 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.tez.client.TezClient;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.app.launcher.TezTestServiceNoOpContainerLauncher;
import org.apache.tez.dag.app.rm.TezTestServiceTaskSchedulerService;
import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
import org.apache.tez.examples.HashJoinExample;
import org.apache.tez.examples.JoinDataGen;
-import org.apache.tez.examples.JoinValidate;
+import org.apache.tez.examples.JoinValidateConfigured;
import org.apache.tez.service.MiniTezTestServiceCluster;
import org.apache.tez.test.MiniTezCluster;
import org.junit.AfterClass;
@@ -47,23 +48,31 @@ public class TestExternalTezServices {
private static final String EXT_PUSH_ENTITY_NAME = "ExtServiceTestPush";
- private static MiniTezCluster tezCluster;
- private static MiniDFSCluster dfsCluster;
- private static MiniTezTestServiceCluster tezTestServiceCluster;
+ private static volatile MiniTezCluster tezCluster;
+ private static volatile MiniDFSCluster dfsCluster;
+ private static volatile MiniTezTestServiceCluster tezTestServiceCluster;
- private static Configuration clusterConf = new Configuration();
- private static Configuration confForJobs;
+ private static volatile Configuration clusterConf = new Configuration();
+ private static volatile Configuration confForJobs;
- private static FileSystem remoteFs;
- private static FileSystem localFs;
+ private static volatile FileSystem remoteFs;
+ private static volatile FileSystem localFs;
- private static TezClient sharedTezClient;
+ private static volatile TezClient sharedTezClient;
+
+ private static final Path SRC_DATA_DIR = new Path("/tmp/" + TestExternalTezServices.class.getSimpleName());
+ private static final Path HASH_JOIN_EXPECTED_RESULT_PATH = new Path(SRC_DATA_DIR, "expectedOutputPath");
+ private static final Path HASH_JOIN_OUTPUT_PATH = new Path(SRC_DATA_DIR, "outPath");
+
+ private static final Map<String, String> PROPS_EXT_SERVICE_PUSH = Maps.newHashMap();
+ private static final Map<String, String> PROPS_REGULAR_CONTAINERS = Maps.newHashMap();
+ private static final Map<String, String> PROPS_IN_AM = Maps.newHashMap();
private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestExternalTezServices.class.getName()
+ "-tmpDir";
@BeforeClass
- public static void setup() throws IOException, TezException, InterruptedException {
+ public static void setup() throws Exception {
localFs = FileSystem.getLocal(clusterConf);
@@ -108,27 +117,79 @@ public class TestExternalTezServices {
remoteFs.mkdirs(stagingDirPath);
// This is currently configured to push tasks into the Service, and then use the standard RPC
confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
- confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
+
+ confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
+// TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskSchedulerService.class.getName());
- confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
- EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
- confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
- EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
+ confForJobs.setStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
+// TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+ EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
+ confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
+// TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
+ EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
- TezConfiguration tezConf = new TezConfiguration(confForJobs);
+ // Default all jobs to run via the service. Individual tests override this on a per vertex/dag level.
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+ confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+
+ // Setup various executor sets
+ PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+ PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+ PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
+
+ PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
+ PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
+ PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
+
+ PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
+ PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
+ PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
+
+
+ // Create a session to use for all tests.
+ TezConfiguration tezClientConf = new TezConfiguration(confForJobs);
sharedTezClient = TezClient.create(TestExternalTezServices.class.getSimpleName() + "_session",
- tezConf, true);
+ tezClientConf, true);
sharedTezClient.start();
LOG.info("Shared TezSession started");
sharedTezClient.waitTillReady();
LOG.info("Shared TezSession ready for submission");
+ // Generate the join data set used for each run.
+ // Can a timeout be enforced here ?
+ remoteFs.mkdirs(SRC_DATA_DIR);
+ Path dataPath1 = new Path(SRC_DATA_DIR, "inPath1");
+ Path dataPath2 = new Path(SRC_DATA_DIR, "inPath2");
+ TezConfiguration tezConf = new TezConfiguration(confForJobs);
+ // Generate join data - with 2 tasks.
+ JoinDataGen dataGen = new JoinDataGen();
+ String[] dataGenArgs = new String[]{
+ dataPath1.toString(), "1048576", dataPath2.toString(), "524288",
+ HASH_JOIN_EXPECTED_RESULT_PATH.toString(), "2"};
+ assertEquals(0, dataGen.run(tezConf, dataGenArgs, sharedTezClient));
+ // Run the actual join - with 2 reducers
+ HashJoinExample joinExample = new HashJoinExample();
+ String[] args = new String[]{
+ dataPath1.toString(), dataPath2.toString(), "2", HASH_JOIN_OUTPUT_PATH.toString()};
+ assertEquals(0, joinExample.run(tezConf, args, sharedTezClient));
+
+ LOG.info("Completed generating Data - Expected Hash Result and Actual Join Result");
}
@AfterClass
@@ -156,35 +217,50 @@ public class TestExternalTezServices {
@Test(timeout = 60000)
- public void test1() throws Exception {
- Path testDir = new Path("/tmp/testHashJoinExample");
+ public void testAllInService() throws Exception {
+ int expectedExternalSubmissions = 4 + 3; //4 for 4 src files, 3 for num reducers.
+ runJoinValidate("AllInService", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
+ PROPS_EXT_SERVICE_PUSH, PROPS_EXT_SERVICE_PUSH);
+ }
- remoteFs.mkdirs(testDir);
+ @Test(timeout = 60000)
+ public void testAllInContainers() throws Exception {
+ int expectedExternalSubmissions = 0; // All in containers
+ runJoinValidate("AllInContainers", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
+ PROPS_REGULAR_CONTAINERS, PROPS_REGULAR_CONTAINERS);
+ }
- Path dataPath1 = new Path(testDir, "inPath1");
- Path dataPath2 = new Path(testDir, "inPath2");
- Path expectedOutputPath = new Path(testDir, "expectedOutputPath");
- Path outPath = new Path(testDir, "outPath");
+ @Test(timeout = 60000)
+ public void testMixed1() throws Exception { // M-ExtService, R-containers
+ int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 3 for num reducers.
+ runJoinValidate("Mixed1", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
+ PROPS_EXT_SERVICE_PUSH, PROPS_REGULAR_CONTAINERS);
+ }
- TezConfiguration tezConf = new TezConfiguration(confForJobs);
+ @Test(timeout = 60000)
+ public void testMixed2() throws Exception { // M-Containers, R-ExtService
+ int expectedExternalSubmissions = 0 + 3; //4 for 4 src files, 3 for num reducers.
+ runJoinValidate("Mixed2", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
+ PROPS_REGULAR_CONTAINERS, PROPS_EXT_SERVICE_PUSH);
+ }
- JoinDataGen dataGen = new JoinDataGen();
- String[] dataGenArgs = new String[]{
- dataPath1.toString(), "1048576", dataPath2.toString(), "524288",
- expectedOutputPath.toString(), "2"};
- assertEquals(0, dataGen.run(tezConf, dataGenArgs, sharedTezClient));
- HashJoinExample joinExample = new HashJoinExample();
- String[] args = new String[]{
- dataPath1.toString(), dataPath2.toString(), "2", outPath.toString()};
- assertEquals(0, joinExample.run(tezConf, args, sharedTezClient));
+ private void runJoinValidate(String name, int extExpectedCount, Map<String, String> lhsProps,
+ Map<String, String> rhsProps,
+ Map<String, String> validateProps) throws
+ Exception {
+ int externalSubmissionCount = tezTestServiceCluster.getNumSubmissions();
- JoinValidate joinValidate = new JoinValidate();
- String[] validateArgs = new String[]{
- expectedOutputPath.toString(), outPath.toString(), "3"};
+ TezConfiguration tezConf = new TezConfiguration(confForJobs);
+ JoinValidateConfigured joinValidate =
+ new JoinValidateConfigured(lhsProps, rhsProps,
+ validateProps, name);
+ String[] validateArgs = new String[]{"-disableSplitGrouping",
+ HASH_JOIN_EXPECTED_RESULT_PATH.toString(), HASH_JOIN_OUTPUT_PATH.toString(), "3"};
assertEquals(0, joinValidate.run(tezConf, validateArgs, sharedTezClient));
// Ensure this was actually submitted to the external cluster
- assertTrue(tezTestServiceCluster.getNumSubmissions() > 0);
+ assertEquals(extExpectedCount,
+ (tezTestServiceCluster.getNumSubmissions() - externalSubmissionCount));
}
}
[22/51] [abbrv] tez git commit: TEZ-2388. Send dag identifier as part
of the fetcher request string. (sseth)
Posted by ss...@apache.org.
TEZ-2388. Send dag identifier as part of the fetcher request string. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/9ed22b2c
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/9ed22b2c
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/9ed22b2c
Branch: refs/heads/TEZ-2003
Commit: 9ed22b2c1e3afaf2e232e5824a29fe662ff71d8e
Parents: 5774527
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Apr 29 08:20:05 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:34 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../tez/runtime/library/common/shuffle/Fetcher.java | 14 ++++++++------
.../runtime/library/common/shuffle/ShuffleUtils.java | 8 +++++---
.../library/common/shuffle/impl/ShuffleManager.java | 2 +-
.../ShuffleInputEventHandlerOrderedGrouped.java | 2 +-
.../runtime/library/common/shuffle/TestFetcher.java | 6 +++---
6 files changed, 19 insertions(+), 14 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/9ed22b2c/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index d42aaf8..9fc9ed3 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -19,5 +19,6 @@ ALL CHANGES:
TEZ-2347. Expose additional information in TaskCommunicatorContext.
TEZ-2361. Propagate dag completion to TaskCommunicator.
TEZ-2381. Fixes after rebase 04/28.
+ TEZ-2388. Send dag identifier as part of the fetcher request string.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/9ed22b2c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
index 08b59ed..1092685 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
@@ -89,6 +89,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
private final FetcherCallback fetcherCallback;
private final FetchedInputAllocator inputManager;
private final ApplicationId appId;
+ private final int dagIdentifier;
private final String logIdentifier;
@@ -130,7 +131,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
private final boolean isDebugEnabled = LOG.isDebugEnabled();
private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
- FetchedInputAllocator inputManager, ApplicationId appId,
+ FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier,
JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf,
RawLocalFileSystem localFs,
LocalDirAllocator localDirAllocator,
@@ -144,6 +145,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
this.inputManager = inputManager;
this.jobTokenSecretMgr = jobTokenSecretManager;
this.appId = appId;
+ this.dagIdentifier = dagIdentifier;
this.pathToAttemptMap = new HashMap<String, InputAttemptIdentifier>();
this.httpConnectionParams = params;
this.conf = conf;
@@ -417,7 +419,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
private HostFetchResult setupConnection(Collection<InputAttemptIdentifier> attempts) {
try {
StringBuilder baseURI = ShuffleUtils.constructBaseURIForShuffleHandler(host,
- port, partition, appId.toString(), httpConnectionParams.isSslShuffle());
+ port, partition, appId.toString(), dagIdentifier, httpConnectionParams.isSslShuffle());
this.url = ShuffleUtils.constructInputURL(baseURI.toString(), attempts,
httpConnectionParams.isKeepAlive());
@@ -930,22 +932,22 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
public FetcherBuilder(FetcherCallback fetcherCallback,
HttpConnectionParams params, FetchedInputAllocator inputManager,
- ApplicationId appId, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
+ ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
Configuration conf, boolean localDiskFetchEnabled, String localHostname, int shufflePort,
boolean asyncHttp) {
- this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
+ this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier,
jobTokenSecretMgr, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled,
false, localHostname, shufflePort, asyncHttp);
}
public FetcherBuilder(FetcherCallback fetcherCallback,
HttpConnectionParams params, FetchedInputAllocator inputManager,
- ApplicationId appId, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
+ ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
Configuration conf, RawLocalFileSystem localFs,
LocalDirAllocator localDirAllocator, Path lockPath,
boolean localDiskFetchEnabled, boolean sharedFetchEnabled,
String localHostname, int shufflePort, boolean asyncHttp) {
- this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
+ this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier,
jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9ed22b2c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
index 1081587..c7cc907 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
@@ -184,19 +184,21 @@ public class ShuffleUtils {
// TODO NEWTEZ handle ssl shuffle
public static StringBuilder constructBaseURIForShuffleHandler(String host,
- int port, int partition, String appId, boolean sslShuffle) {
+ int port, int partition, String appId, int dagIdentifier, boolean sslShuffle) {
return constructBaseURIForShuffleHandler(host + ":" + String.valueOf(port),
- partition, appId, sslShuffle);
+ partition, appId, dagIdentifier, sslShuffle);
}
public static StringBuilder constructBaseURIForShuffleHandler(String hostIdentifier,
- int partition, String appId, boolean sslShuffle) {
+ int partition, String appId, int dagIdentifier, boolean sslShuffle) {
final String http_protocol = (sslShuffle) ? "https://" : "http://";
StringBuilder sb = new StringBuilder(http_protocol);
sb.append(hostIdentifier);
sb.append("/");
sb.append("mapOutput?job=");
sb.append(appId.replace("application", "job"));
+ sb.append("&dag=");
+ sb.append(String.valueOf(dagIdentifier));
sb.append("&reduce=");
sb.append(String.valueOf(partition));
sb.append("&map=");
http://git-wip-us.apache.org/repos/asf/tez/blob/9ed22b2c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
index 600c332..5bfcab8 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
@@ -403,7 +403,7 @@ public class ShuffleManager implements FetcherCallback {
}
FetcherBuilder fetcherBuilder = new FetcherBuilder(ShuffleManager.this,
- httpConnectionParams, inputManager, inputContext.getApplicationId(),
+ httpConnectionParams, inputManager, inputContext.getApplicationId(), inputContext.getDagIdentifier(),
jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
lockDisk, localDiskFetchEnabled, sharedFetchEnabled,
localhostName, shufflePort, asyncHttp);
http://git-wip-us.apache.org/repos/asf/tez/blob/9ed22b2c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
index 32ac766..9481e65 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
@@ -127,7 +127,7 @@ public class ShuffleInputEventHandlerOrderedGrouped {
@VisibleForTesting
URI getBaseURI(String host, int port, int partitionId) {
StringBuilder sb = ShuffleUtils.constructBaseURIForShuffleHandler(host, port,
- partitionId, inputContext.getApplicationId().toString(), sslShuffle);
+ partitionId, inputContext.getApplicationId().toString(), inputContext.getDagIdentifier(), sslShuffle);
URI u = URI.create(sb.toString());
return u;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/9ed22b2c/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
index 7678b18..85e3540 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
@@ -75,7 +75,7 @@ public class TestFetcher {
final boolean DISABLE_LOCAL_FETCH = false;
Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
- ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
+ ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
PORT, false);
builder.assignWork(HOST, PORT, 0, Arrays.asList(srcAttempts));
Fetcher fetcher = spy(builder.build());
@@ -125,7 +125,7 @@ public class TestFetcher {
// When disabled use http fetch
conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false);
builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null,
- ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, DISABLE_LOCAL_FETCH, HOST,
+ ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, DISABLE_LOCAL_FETCH, HOST,
PORT, false);
builder.assignWork(HOST, PORT, 0, Arrays.asList(srcAttempts));
fetcher = spy(builder.build());
@@ -159,7 +159,7 @@ public class TestFetcher {
int partition = 42;
FetcherCallback callback = mock(FetcherCallback.class);
Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(callback, null, null,
- ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, true, HOST, PORT, false);
+ ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, false);
builder.assignWork(HOST, PORT, partition, Arrays.asList(srcAttempts));
Fetcher fetcher = spy(builder.build());
[37/51] [abbrv] tez git commit: TEZ-2004. Define basic interface for
pluggable ContainerLaunchers. (sseth)
Posted by ss...@apache.org.
TEZ-2004. Define basic interface for pluggable ContainerLaunchers. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/d799d3b9
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/d799d3b9
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/d799d3b9
Branch: refs/heads/TEZ-2003
Commit: d799d3b9397f716274265c4bf4a24b2003eeaa83
Parents: 8a03f11
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Jul 20 15:52:24 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../serviceplugins/api/ContainerEndReason.java | 31 +++++
.../api/ContainerLaunchRequest.java | 81 +++++++++++
.../serviceplugins/api/ContainerLauncher.java | 46 +++++++
.../api/ContainerLauncherContext.java | 54 ++++++++
.../api/ContainerLauncherOperationBase.java | 58 ++++++++
.../api/ContainerStopRequest.java | 47 +++++++
.../api/TaskAttemptEndReason.java | 32 +++++
.../org/apache/tez/common/TezUtilsInternal.java | 9 +-
.../apache/tez/dag/api/ContainerEndReason.java | 27 ----
.../tez/dag/api/TaskAttemptEndReason.java | 27 ----
.../apache/tez/dag/api/TaskCommunicator.java | 9 ++
.../tez/dag/api/TaskCommunicatorContext.java | 2 +-
.../tez/dag/api/TaskCommunicatorInterface.java | 18 +++
.../java/org/apache/tez/dag/app/AppContext.java | 5 +
.../dag/app/ContainerLauncherContextImpl.java | 101 ++++++++++++++
.../org/apache/tez/dag/app/DAGAppMaster.java | 15 +++
.../apache/tez/dag/app/TaskAttemptListener.java | 6 +-
.../dag/app/TaskAttemptListenerImpTezDag.java | 4 +-
.../dag/app/TaskCommunicatorContextImpl.java | 6 +-
.../tez/dag/app/TezTaskCommunicatorImpl.java | 12 +-
.../tez/dag/app/launcher/ContainerLauncher.java | 29 ----
.../dag/app/launcher/ContainerLauncherImpl.java | 128 ++++++++----------
.../app/launcher/ContainerLauncherRouter.java | 52 +++++++-
.../tez/dag/app/launcher/ContainerOp.java | 62 +++++++++
.../app/launcher/LocalContainerLauncher.java | 123 ++++++++---------
.../tez/dag/app/rm/AMSchedulerEventTAEnded.java | 2 +-
.../dag/app/rm/LocalTaskSchedulerService.java | 2 +-
.../tez/dag/app/rm/NMCommunicatorEvent.java | 18 ++-
.../rm/NMCommunicatorLaunchRequestEvent.java | 11 +-
.../app/rm/NMCommunicatorStopRequestEvent.java | 4 +-
.../tez/dag/app/rm/TaskSchedulerService.java | 2 +-
.../dag/app/rm/YarnTaskSchedulerService.java | 2 +-
.../rm/container/AMContainerEventCompleted.java | 2 +-
.../dag/app/rm/container/AMContainerImpl.java | 9 +-
.../apache/tez/dag/app/MockDAGAppMaster.java | 63 ++++-----
.../app/TestTaskAttemptListenerImplTezDag.java | 4 +-
.../app/TestTaskAttemptListenerImplTezDag2.java | 2 +-
.../tez/dag/app/rm/TestContainerReuse.java | 4 +-
.../app/rm/TestLocalTaskSchedulerService.java | 1 -
.../dag/app/rm/container/TestAMContainer.java | 4 +-
.../TezTestServiceContainerLauncher.java | 133 +++++++++----------
.../TezTestServiceNoOpContainerLauncher.java | 53 +++-----
.../rm/TezTestServiceTaskSchedulerService.java | 2 +-
.../TezTestServiceTaskCommunicatorImpl.java | 4 +-
45 files changed, 887 insertions(+), 420 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 604947c..88dd0c7 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -34,5 +34,6 @@ ALL CHANGES:
TEZ-2526. Fix version for tez-history-parser.
TEZ-2621. rebase 07/14
TEZ-2124. Change Node tracking to work per external container source.
+ TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerEndReason.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerEndReason.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerEndReason.java
new file mode 100644
index 0000000..ab8619f
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerEndReason.java
@@ -0,0 +1,31 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum ContainerEndReason {
+ NODE_FAILED, // Completed because the node running the container was marked as dead
+ INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
+ EXTERNAL_PREEMPTION, // Preempted due to cluster contention
+ APPLICATION_ERROR, // An error in the AM caused by user code
+ FRAMEWORK_ERROR, // An error in the AM - likely a bug.
+ LAUNCH_FAILED, // Failure to launch the container
+ COMPLETED, // Completed via normal flow
+ OTHER
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
new file mode 100644
index 0000000..cfd7ca7
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLaunchRequest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerLaunchRequest extends ContainerLauncherOperationBase {
+
+ private final ContainerLaunchContext clc;
+ private final Container container;
+ private final String schedulerName;
+ private final String taskCommName;
+
+ public ContainerLaunchRequest(NodeId nodeId,
+ ContainerId containerId,
+ Token containerToken,
+ ContainerLaunchContext clc,
+ Container container, String schedulerName, String taskCommName) {
+ super(nodeId, containerId, containerToken);
+ this.clc = clc;
+ this.container = container;
+ this.schedulerName = schedulerName;
+ this.taskCommName = taskCommName;
+ }
+
+
+ // TODO Post TEZ-2003. TEZ-2625. ContainerLaunchContext needs to be built here instead of being passed in.
+ // Basic specifications need to be provided here
+ public ContainerLaunchContext getContainerLaunchContext() {
+ return clc;
+ }
+
+ /**
+ * Get the name of the task communicator which will be used to communicate
+ * with the task that will run in this container.
+ * @return
+ */
+ public String getTaskCommunicatorName() {
+ return taskCommName;
+ }
+
+ /**
+ * Get the name of the scheduler which allocated this container.
+ * @return
+ */
+ public String getSchedulerName() {
+ return schedulerName;
+ }
+
+ @Override
+ public String toString() {
+ return "ContainerLaunchRequest{" +
+ "nodeId=" + getNodeId() +
+ ", containerId=" + getContainerId() +
+ ", clc=" + clc +
+ ", container=" + container +
+ ", schedulerName='" + schedulerName + '\'' +
+ ", taskCommName='" + taskCommName + '\'' +
+ '}';
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
new file mode 100644
index 0000000..218edb6
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
@@ -0,0 +1,46 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.AbstractService;
+
+/**
+ * Plugin to allow custom container launchers to be written to launch containers on different types
+ * of executors.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class ContainerLauncher extends AbstractService {
+
+ private final ContainerLauncherContext containerLauncherContext;
+
+ // TODO TEZ-2003 Simplify this by moving away from AbstractService. Potentially Guava AbstractService.
+ // A serviceInit(Configuration) is not likely to be very useful, and will expose unnecessary internal
+ // configuration to the services if populated with the AM Configuration
+ public ContainerLauncher(String name, ContainerLauncherContext containerLauncherContext) {
+ super(name);
+ this.containerLauncherContext = containerLauncherContext;
+ }
+
+ public final ContainerLauncherContext getContext() {
+ return this.containerLauncherContext;
+ }
+
+ public abstract void launchContainer(ContainerLaunchRequest launchRequest);
+ public abstract void stopContainer(ContainerStopRequest stopRequest);
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
new file mode 100644
index 0000000..836dc4a
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface ContainerLauncherContext {
+
+ // TODO Post TEZ-2003. Tez abstraction for ContainerId, NodeId, other YARN constructs
+
+ // Reporting APIs
+ void containerLaunched(ContainerId containerId);
+
+ void containerLaunchFailed(ContainerId containerId, String diagnostics);
+
+ void containerStopRequested(ContainerId containerId);
+
+ void containerStopFailed(ContainerId containerId, String diagnostics);
+
+ // TODO Post TEZ-2003. TaskAttemptEndReason does not belong here, and is an unnecessary leak.
+ // ContainerCompleted is normally generated by the scheduler in case of YARN since the RM informs about completion.
+ // For other sources, there may not be a central entity making this information available. The ContainerLauncher
+ // on the stop request will likely be the best place to generate it.
+ void containerCompleted(ContainerId containerId, int exitStatus, String diagnostics, TaskAttemptEndReason endReason);
+
+ // Lookup APIs
+
+ // TODO TEZ-2003. To be replaced by getInitialPayload once the DAG API is changed.
+ Configuration getInitialConfiguration();
+
+ int getNumNodes(String sourceName);
+
+ ApplicationAttemptId getApplicationAttemptId();
+
+ Object getTaskCommunicatorMetaInfo(String taskCommName);
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
new file mode 100644
index 0000000..29e0420
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherOperationBase.java
@@ -0,0 +1,58 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerLauncherOperationBase {
+
+ private final NodeId nodeId;
+ private final ContainerId containerId;
+ private final Token containerToken;
+
+ public ContainerLauncherOperationBase(NodeId nodeId,
+ ContainerId containerId,
+ Token containerToken) {
+ this.nodeId = nodeId;
+ this.containerId = containerId;
+ this.containerToken = containerToken;
+ }
+
+ public NodeId getNodeId() {
+ return nodeId;
+ }
+
+ public ContainerId getContainerId() {
+ return containerId;
+ }
+
+ public Token getContainerToken() {
+ return containerToken;
+ }
+
+ @Override
+ public String toString() {
+ return "ContainerLauncherOperationBase{" +
+ "nodeId=" + nodeId +
+ ", containerId=" + containerId +
+ '}';
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
new file mode 100644
index 0000000..cb0af31
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerStopRequest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ContainerStopRequest extends ContainerLauncherOperationBase {
+
+ private final String schedulerName;
+ private final String taskCommName;
+
+ public ContainerStopRequest(NodeId nodeId,
+ ContainerId containerId,
+ Token containerToken, String schedulerName, String taskCommName) {
+ super(nodeId, containerId, containerToken);
+ this.schedulerName = schedulerName;
+ this.taskCommName = taskCommName;
+ }
+
+ @Override
+ public String toString() {
+ return "ContainerStopRequest{" +
+ "nodeId=" + getNodeId() +
+ ", containerId=" + getContainerId() +
+ ", schedulerName='" + schedulerName + '\'' +
+ ", taskCommName='" + taskCommName + '\'' +
+ '}';
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
new file mode 100644
index 0000000..4255c28
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskAttemptEndReason.java
@@ -0,0 +1,32 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum TaskAttemptEndReason {
+ NODE_FAILED, // Completed because the node running the container was marked as dead
+ COMMUNICATION_ERROR, // Communication error with the task
+ SERVICE_BUSY, // External service busy
+ INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
+ EXTERNAL_PREEMPTION, // Preempted due to cluster contention
+ APPLICATION_ERROR, // An error in the AM caused by user code
+ FRAMEWORK_ERROR, // An error in the AM - likely a bug.
+ CONTAINER_EXITED,
+ OTHER // Unknown reason
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
index 0bdeb79..4c8c227 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
@@ -35,13 +35,12 @@ import java.util.zip.Inflater;
import com.google.protobuf.ByteString;
import com.google.protobuf.Descriptors;
import com.google.protobuf.TextFormat;
-import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.log4j.Appender;
import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.records.DAGProtos;
import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
@@ -256,6 +255,8 @@ public class TezUtilsInternal {
return TaskAttemptTerminationCause.FRAMEWORK_ERROR;
case NODE_FAILED:
return TaskAttemptTerminationCause.NODE_FAILED;
+ case CONTAINER_EXITED:
+ return TaskAttemptTerminationCause.CONTAINER_EXITED;
case OTHER:
return TaskAttemptTerminationCause.UNKNOWN_ERROR;
default:
@@ -283,6 +284,8 @@ public class TezUtilsInternal {
return TaskAttemptEndReason.FRAMEWORK_ERROR;
case NODE_FAILED:
return TaskAttemptEndReason.NODE_FAILED;
+ case CONTAINER_EXITED:
+ return TaskAttemptEndReason.CONTAINER_EXITED;
case INTERRUPTED_BY_SYSTEM:
case INTERRUPTED_BY_USER:
case UNKNOWN_ERROR:
@@ -296,7 +299,7 @@ public class TezUtilsInternal {
case OUTPUT_LOST:
case TASK_HEARTBEAT_ERROR:
case CONTAINER_LAUNCH_FAILED:
- case CONTAINER_EXITED:
+
case CONTAINER_STOPPED:
case NODE_DISK_ERROR:
default:
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
deleted file mode 100644
index e13e886..0000000
--- a/tez-common/src/main/java/org/apache/tez/dag/api/ContainerEndReason.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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.tez.dag.api;
-
-// TODO TEZ-2003 Expose as a public API
-public enum ContainerEndReason {
- NODE_FAILED, // Completed because the node running the container was marked as dead
- INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
- EXTERNAL_PREEMPTION, // Preempted due to cluster contention
- APPLICATION_ERROR, // An error in the AM caused by user code
- FRAMEWORK_ERROR, // An error in the AM - likely a bug.
- LAUNCH_FAILED, // Failure to launch the container
- COMPLETED, // Completed via normal flow
- OTHER
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
deleted file mode 100644
index de78d21..0000000
--- a/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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.tez.dag.api;
-
-// TODO TEZ-2003 Expose as a public API
-public enum TaskAttemptEndReason {
- NODE_FAILED, // Completed because the node running the container was marked as dead
- COMMUNICATION_ERROR, // Communication error with the task
- SERVICE_BUSY, // External service busy
- INTERNAL_PREEMPTION, // Preempted by the AM, due to an internal decision
- EXTERNAL_PREEMPTION, // Preempted due to cluster contention
- APPLICATION_ERROR, // An error in the AM caused by user code
- FRAMEWORK_ERROR, // An error in the AM - likely a bug.
- OTHER // Unknown reason
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index d0a006b..05e437c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -22,6 +22,8 @@ import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.api.impl.TaskSpec;
@@ -87,4 +89,11 @@ public abstract class TaskCommunicator extends AbstractService {
// TODO TEZ-2003 This is extremely difficult to use. Add the dagStarted notification, and potentially
// throw exceptions between a dagComplete and dagStart invocation.
public abstract void dagComplete(String dagName);
+
+ /**
+ * Share meta-information such as host:port information where the Task Communicator may be listening.
+ * Primarily for use by compatible launchers to learn this information.
+ * @return
+ */
+ public abstract Object getMetaInfo();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 56345ab..b6e63f7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -16,13 +16,13 @@ package org.apache.tez.dag.api;
import javax.annotation.Nullable;
import java.io.IOException;
-import java.util.Collection;
import java.util.Set;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.records.TezTaskAttemptID;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
new file mode 100644
index 0000000..022cd7b
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
@@ -0,0 +1,18 @@
+/*
+ * 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.tez.dag.api;
+
+public interface TaskCommunicatorInterface {
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
index 1ccb10b..516fcef 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
@@ -118,4 +118,9 @@ public interface AppContext {
public Integer getTaskCommunicatorIdentifier(String name);
public Integer getTaskScheduerIdentifier(String name);
public Integer getContainerLauncherIdentifier(String name);
+
+ public String getTaskCommunicatorName(int taskCommId);
+ public String getTaskSchedulerName(int schedulerId);
+ public String getContainerLauncherName(int launcherId);
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
new file mode 100644
index 0000000..997775a
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
@@ -0,0 +1,101 @@
+/*
+ * 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.tez.dag.app;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunchFailed;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventStopFailed;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+
+public class ContainerLauncherContextImpl implements ContainerLauncherContext {
+
+ private final AppContext context;
+ private final TaskAttemptListener tal;
+
+ public ContainerLauncherContextImpl(AppContext appContext, TaskAttemptListener tal) {
+ this.context = appContext;
+ this.tal = tal;
+ }
+
+ @Override
+ public void containerLaunched(ContainerId containerId) {
+ context.getEventHandler().handle(
+ new AMContainerEventLaunched(containerId));
+ ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
+ containerId, context.getClock().getTime(), context.getApplicationAttemptId());
+ context.getHistoryHandler().handle(new DAGHistoryEvent(
+ null, lEvt));
+
+ }
+
+ @Override
+ public void containerLaunchFailed(ContainerId containerId, String diagnostics) {
+ context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, diagnostics));
+ }
+
+ @Override
+ public void containerStopRequested(ContainerId containerId) {
+ context.getEventHandler().handle(
+ new AMContainerEvent(containerId, AMContainerEventType.C_NM_STOP_SENT));
+ }
+
+ @Override
+ public void containerStopFailed(ContainerId containerId, String diagnostics) {
+ context.getEventHandler().handle(
+ new AMContainerEventStopFailed(containerId, diagnostics));
+ }
+
+ @Override
+ public void containerCompleted(ContainerId containerId, int exitStatus, String diagnostics,
+ TaskAttemptEndReason endReason) {
+ context.getEventHandler().handle(new AMContainerEventCompleted(containerId, exitStatus, diagnostics, TezUtilsInternal
+ .fromTaskAttemptEndReason(
+ endReason)));
+ }
+
+ @Override
+ public Configuration getInitialConfiguration() {
+ return context.getAMConf();
+ }
+
+ @Override
+ public int getNumNodes(String sourceName) {
+ int sourceIndex = context.getTaskScheduerIdentifier(sourceName);
+ int numNodes = context.getNodeTracker().getNumNodes(sourceIndex);
+ return numNodes;
+ }
+
+ @Override
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return context.getApplicationAttemptId();
+ }
+
+ @Override
+ public Object getTaskCommunicatorMetaInfo(String taskCommName) {
+ int taskCommId = context.getTaskCommunicatorIdentifier(taskCommName);
+ return tal.getTaskCommunicator(taskCommId).getMetaInfo();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 04e72db..d56fb95 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -1551,6 +1551,21 @@ public class DAGAppMaster extends AbstractService {
}
@Override
+ public String getTaskCommunicatorName(int taskCommId) {
+ return taskCommunicators.inverse().get(taskCommId);
+ }
+
+ @Override
+ public String getTaskSchedulerName(int schedulerId) {
+ return taskSchedulers.inverse().get(schedulerId);
+ }
+
+ @Override
+ public String getContainerLauncherName(int launcherId) {
+ return containerLaunchers.inverse().get(launcherId);
+ }
+
+ @Override
public Map<ApplicationAccessType, String> getApplicationACLs() {
if (getServiceState() != STATE.STARTED) {
throw new TezUncheckedException(
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
index 92e38ae..2eec2fb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
@@ -18,11 +18,9 @@
package org.apache.tez.dag.app;
-import java.net.InetSocketAddress;
-
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.app.rm.container.AMContainerTask;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index e2d44e2..47b63dd 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -28,7 +28,7 @@ import java.util.concurrent.ConcurrentMap;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.collections4.ListUtils;
-import org.apache.tez.dag.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
@@ -43,7 +43,7 @@ import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TaskCommunicatorContext;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index 790066f..50e006d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -17,10 +17,6 @@ package org.apache.tez.dag.app;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import com.google.common.base.Function;
@@ -30,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
import org.apache.tez.dag.api.TaskHeartbeatResponse;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 83322f2..0374022 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -41,8 +41,8 @@ import org.apache.tez.common.ContainerContext;
import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.common.security.TokenCache;
-import org.apache.tez.dag.api.ContainerEndReason;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -180,7 +180,8 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
@Override
public void registerRunningContainer(ContainerId containerId, String host, int port) {
- ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId, new ContainerInfo(containerId, host, port));
+ ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId,
+ new ContainerInfo(containerId, host, port));
if (oldInfo != null) {
throw new TezUncheckedException("Multiple registrations for containerId: " + containerId);
}
@@ -267,6 +268,11 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
// Nothing to do at the moment. Some of the TODOs from TaskAttemptListener apply here.
}
+ @Override
+ public Object getMetaInfo() {
+ return address;
+ }
+
protected String getTokenIdentifier() {
return tokenIdentifier;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
deleted file mode 100644
index ea07a1d..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncher.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements. See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership. The ASF licenses this file
-* to you 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.tez.dag.app.launcher;
-
-
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tez.dag.app.dag.DAG;
-import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
-
-public interface ContainerLauncher
- extends EventHandler<NMCommunicatorEvent> {
-
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
index a12fb04..fe0178c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
@@ -30,12 +30,15 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
import org.apache.tez.dag.api.TezConstants;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -45,57 +48,43 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
-import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.Records;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
-import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEventLaunchFailed;
-import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
-import org.apache.tez.dag.app.rm.container.AMContainerEventStopFailed;
-import org.apache.tez.dag.app.rm.container.AMContainerEventType;
-import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-// TODO XXX: See what part of this lifecycle and state management can be simplified.
+// TODO See what part of this lifecycle and state management can be simplified.
// Ideally, no state - only sendStart / sendStop.
-// TODO XXX: Review this entire code and clean it up.
+// TODO Review this entire code and clean it up.
/**
* This class is responsible for launching of containers.
*/
-public class ContainerLauncherImpl extends AbstractService implements
- ContainerLauncher {
+public class ContainerLauncherImpl extends ContainerLauncher {
- // TODO XXX Ensure the same thread is used to launch / stop the same container. Or - ensure event ordering.
+ // TODO Ensure the same thread is used to launch / stop the same container. Or - ensure event ordering.
static final Logger LOG = LoggerFactory.getLogger(ContainerLauncherImpl.class);
- private ConcurrentHashMap<ContainerId, Container> containers =
- new ConcurrentHashMap<ContainerId, Container>();
- private AppContext context;
+ private final ConcurrentHashMap<ContainerId, Container> containers =
+ new ConcurrentHashMap<>();
protected ThreadPoolExecutor launcherPool;
protected static final int INITIAL_POOL_SIZE = 10;
- private int limitOnPoolSize;
+ private final int limitOnPoolSize;
+ private final Configuration conf;
private Thread eventHandlingThread;
- protected BlockingQueue<NMCommunicatorEvent> eventQueue =
- new LinkedBlockingQueue<NMCommunicatorEvent>();
- private Clock clock;
+ protected BlockingQueue<ContainerOp> eventQueue = new LinkedBlockingQueue<>();
private ContainerManagementProtocolProxy cmProxy;
private AtomicBoolean serviceStopped = new AtomicBoolean(false);
- private Container getContainer(NMCommunicatorEvent event) {
- ContainerId id = event.getContainerId();
+ private Container getContainer(ContainerOp event) {
+ ContainerId id = event.getBaseOperation().getContainerId();
Container c = containers.get(id);
if(c == null) {
- c = new Container(event.getContainerId(),
- event.getNodeId().toString(), event.getContainerToken());
+ c = new Container(id,
+ event.getBaseOperation().getNodeId().toString(), event.getBaseOperation().getContainerToken());
Container old = containers.putIfAbsent(id, c);
if(old != null) {
c = old;
@@ -111,6 +100,7 @@ public class ContainerLauncherImpl extends AbstractService implements
}
}
+
private static enum ContainerState {
PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
}
@@ -135,7 +125,7 @@ public class ContainerLauncherImpl extends AbstractService implements
}
@SuppressWarnings("unchecked")
- public synchronized void launch(NMCommunicatorLaunchRequestEvent event) {
+ public synchronized void launch(ContainerLaunchRequest event) {
LOG.info("Launching Container with Id: " + event.getContainerId());
if(this.state == ContainerState.KILLED_BEFORE_LAUNCH) {
state = ContainerState.DONE;
@@ -171,13 +161,7 @@ public class ContainerLauncherImpl extends AbstractService implements
// after launching, send launched event to task attempt to move
// it from ASSIGNED to RUNNING state
- context.getEventHandler().handle(
- new AMContainerEventLaunched(containerID));
- ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
- containerID, clock.getTime(), context.getApplicationAttemptId());
- context.getHistoryHandler().handle(new DAGHistoryEvent(
- null, lEvt));
-
+ getContext().containerLaunched(containerID);
this.state = ContainerState.RUNNING;
} catch (Throwable t) {
String message = "Container launch failed for " + containerID + " : "
@@ -217,16 +201,14 @@ public class ContainerLauncherImpl extends AbstractService implements
// If stopContainer returns without an error, assuming the stop made
// it over to the NodeManager.
- context.getEventHandler().handle(
- new AMContainerEvent(containerID, AMContainerEventType.C_NM_STOP_SENT));
+ getContext().containerStopRequested(containerID);
} catch (Throwable t) {
// ignore the cleanup failure
String message = "cleanup failed for container "
+ this.containerID + " : "
+ ExceptionUtils.getStackTrace(t);
- context.getEventHandler().handle(
- new AMContainerEventStopFailed(containerID, message));
+ getContext().containerStopFailed(containerID, message);
LOG.warn(message);
this.state = ContainerState.DONE;
return;
@@ -240,15 +222,9 @@ public class ContainerLauncherImpl extends AbstractService implements
}
}
- public ContainerLauncherImpl(AppContext context) {
- super(ContainerLauncherImpl.class.getName());
- this.context = context;
- this.clock = context.getClock();
- }
-
- @Override
- public synchronized void serviceInit(Configuration config) {
- Configuration conf = new Configuration(config);
+ public ContainerLauncherImpl(ContainerLauncherContext containerLauncherContext) {
+ super(ContainerLauncherImpl.class.getName(), containerLauncherContext);
+ this.conf = new Configuration(containerLauncherContext.getInitialConfiguration());
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
0);
@@ -262,7 +238,7 @@ public class ContainerLauncherImpl extends AbstractService implements
public void serviceStart() {
// pass a copy of config to ContainerManagementProtocolProxy until YARN-3497 is fixed
cmProxy =
- new ContainerManagementProtocolProxy(new Configuration(getConfig()));
+ new ContainerManagementProtocolProxy(conf);
ThreadFactory tf = new ThreadFactoryBuilder().setNameFormat(
"ContainerLauncher #%d").setDaemon(true).build();
@@ -275,7 +251,7 @@ public class ContainerLauncherImpl extends AbstractService implements
eventHandlingThread = new Thread() {
@Override
public void run() {
- NMCommunicatorEvent event = null;
+ ContainerOp event = null;
while (!Thread.currentThread().isInterrupted()) {
try {
event = eventQueue.take();
@@ -293,9 +269,8 @@ public class ContainerLauncherImpl extends AbstractService implements
// nodes where containers will run at *this* point of time. This is
// *not* the cluster size and doesn't need to be.
- int yarnSourceIndex =
- context.getTaskScheduerIdentifier(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
- int numNodes = context.getNodeTracker().getNumNodes(yarnSourceIndex);
+ int numNodes = getContext().getNumNodes(
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
int idealPoolSize = Math.min(limitOnPoolSize, numNodes);
if (poolSize < idealPoolSize) {
@@ -347,7 +322,7 @@ public class ContainerLauncherImpl extends AbstractService implements
}
}
- protected EventProcessor createEventProcessor(NMCommunicatorEvent event) {
+ protected EventProcessor createEventProcessor(ContainerOp event) {
return new EventProcessor(event);
}
@@ -361,32 +336,29 @@ public class ContainerLauncherImpl extends AbstractService implements
* Setup and start the container on remote nodemanager.
*/
class EventProcessor implements Runnable {
- private NMCommunicatorEvent event;
+ private ContainerOp event;
- EventProcessor(NMCommunicatorEvent event) {
+ EventProcessor(ContainerOp event) {
this.event = event;
}
@Override
public void run() {
- LOG.info("Processing the event " + event.toString());
+ LOG.info("Processing operation {}", event.toString());
// Load ContainerManager tokens before creating a connection.
// TODO: Do it only once per NodeManager.
- ContainerId containerID = event.getContainerId();
+ ContainerId containerID = event.getBaseOperation().getContainerId();
Container c = getContainer(event);
- switch(event.getType()) {
-
- case CONTAINER_LAUNCH_REQUEST:
- NMCommunicatorLaunchRequestEvent launchEvent
- = (NMCommunicatorLaunchRequestEvent) event;
- c.launch(launchEvent);
- break;
-
- case CONTAINER_STOP_REQUEST:
- c.kill();
- break;
+ switch(event.getOpType()) {
+ case LAUNCH_REQUEST:
+ ContainerLaunchRequest launchRequest = event.getLaunchRequest();
+ c.launch(launchRequest);
+ break;
+ case STOP_REQUEST:
+ c.kill();
+ break;
}
removeContainerIfDone(containerID);
}
@@ -408,13 +380,23 @@ public class ContainerLauncherImpl extends AbstractService implements
void sendContainerLaunchFailedMsg(ContainerId containerId,
String message) {
LOG.error(message);
- context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, message));
+ getContext().containerLaunchFailed(containerId, message);
+ }
+
+
+ @Override
+ public void launchContainer(ContainerLaunchRequest launchRequest) {
+ try {
+ eventQueue.put(new ContainerOp(ContainerOp.OPType.LAUNCH_REQUEST, launchRequest));
+ } catch (InterruptedException e) {
+ throw new TezUncheckedException(e);
+ }
}
@Override
- public void handle(NMCommunicatorEvent event) {
+ public void stopContainer(ContainerStopRequest stopRequest) {
try {
- eventQueue.put(event);
+ eventQueue.put(new ContainerOp(ContainerOp.OPType.STOP_REQUEST, stopRequest));
} catch (InterruptedException e) {
throw new TezUncheckedException(e);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index db145f4..9f741cf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -23,12 +23,18 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ContainerLauncherContextImpl;
import org.apache.tez.dag.app.TaskAttemptListener;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -38,11 +44,15 @@ public class ContainerLauncherRouter extends AbstractService
static final Logger LOG = LoggerFactory.getLogger(ContainerLauncherImpl.class);
private final ContainerLauncher containerLaunchers[];
+ private final ContainerLauncherContext containerLauncherContexts[];
+ private final AppContext appContext;
@VisibleForTesting
- public ContainerLauncherRouter(ContainerLauncher containerLauncher) {
+ public ContainerLauncherRouter(ContainerLauncher containerLauncher, AppContext context) {
super(ContainerLauncherRouter.class.getName());
+ this.appContext = context;
containerLaunchers = new ContainerLauncher[] {containerLauncher};
+ containerLauncherContexts = new ContainerLauncherContext[] {containerLauncher.getContext()};
}
// Accepting conf to setup final parameters, if required.
@@ -53,6 +63,7 @@ public class ContainerLauncherRouter extends AbstractService
boolean isPureLocalMode) throws UnknownHostException {
super(ContainerLauncherRouter.class.getName());
+ this.appContext = context;
if (containerLauncherClassIdentifiers == null || containerLauncherClassIdentifiers.length == 0) {
if (isPureLocalMode) {
containerLauncherClassIdentifiers =
@@ -62,16 +73,21 @@ public class ContainerLauncherRouter extends AbstractService
new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
}
}
+ containerLauncherContexts = new ContainerLauncherContext[containerLauncherClassIdentifiers.length];
containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
+
for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
+ ContainerLauncherContext containerLauncherContext = new ContainerLauncherContextImpl(context, taskAttemptListener);
+ containerLauncherContexts[i] = containerLauncherContext;
containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
- taskAttemptListener, workingDirectory, isPureLocalMode, conf);
+ containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode, conf);
}
}
private ContainerLauncher createContainerLauncher(String containerLauncherClassIdentifier,
AppContext context,
+ ContainerLauncherContext containerLauncherContext,
TaskAttemptListener taskAttemptListener,
String workingDirectory,
boolean isPureLocalMode,
@@ -79,12 +95,15 @@ public class ContainerLauncherRouter extends AbstractService
UnknownHostException {
if (containerLauncherClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
LOG.info("Creating DefaultContainerLauncher");
- return new ContainerLauncherImpl(context);
+ return new ContainerLauncherImpl(containerLauncherContext);
} else if (containerLauncherClassIdentifier
.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Creating LocalContainerLauncher");
+ // TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
+ // extensive internals which are only available at runtime. Will likely require
+ // some kind of runtime binding of parameters in the payload to work correctly.
return
- new LocalContainerLauncher(context, taskAttemptListener, workingDirectory, isPureLocalMode);
+ new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener, workingDirectory, isPureLocalMode);
} else {
LOG.info("Creating container launcher : " + containerLauncherClassIdentifier);
Class<? extends ContainerLauncher> containerLauncherClazz =
@@ -92,9 +111,9 @@ public class ContainerLauncherRouter extends AbstractService
containerLauncherClassIdentifier);
try {
Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
- .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
+ .getConstructor(ContainerLauncherContext.class);
ctor.setAccessible(true);
- return ctor.newInstance(context, conf, taskAttemptListener);
+ return ctor.newInstance(containerLauncherContext);
} catch (NoSuchMethodException e) {
throw new TezUncheckedException(e);
} catch (InvocationTargetException e) {
@@ -141,6 +160,25 @@ public class ContainerLauncherRouter extends AbstractService
@Override
public void handle(NMCommunicatorEvent event) {
- containerLaunchers[event.getLauncherId()].handle(event);
+ int launcherId = event.getLauncherId();
+ String schedulerName = appContext.getTaskSchedulerName(event.getSchedulerId());
+ String taskCommName = appContext.getTaskCommunicatorName(event.getTaskCommId());
+ switch (event.getType()) {
+ case CONTAINER_LAUNCH_REQUEST:
+ NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
+ ContainerLaunchRequest launchRequest =
+ new ContainerLaunchRequest(launchEvent.getNodeId(), launchEvent.getContainerId(),
+ launchEvent.getContainerToken(), launchEvent.getContainerLaunchContext(),
+ launchEvent.getContainer(), schedulerName,
+ taskCommName);
+ containerLaunchers[launcherId].launchContainer(launchRequest);
+ break;
+ case CONTAINER_STOP_REQUEST:
+ ContainerStopRequest stopRequest =
+ new ContainerStopRequest(event.getNodeId(), event.getContainerId(),
+ event.getContainerToken(), schedulerName, taskCommName);
+ containerLaunchers[launcherId].stopContainer(stopRequest);
+ break;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java
new file mode 100644
index 0000000..c62de66
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerOp.java
@@ -0,0 +1,62 @@
+/*
+ * 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.tez.dag.app.launcher;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncherOperationBase;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
+
+@InterfaceAudience.Private
+public class ContainerOp {
+ enum OPType {
+ LAUNCH_REQUEST, STOP_REQUEST
+ }
+
+ final ContainerLauncherOperationBase command;
+ final OPType opType;
+
+ public ContainerOp(OPType opType, ContainerLauncherOperationBase command) {
+ this.opType = opType;
+ this.command = command;
+ }
+
+ public OPType getOpType() {
+ return opType;
+ }
+
+ public ContainerLauncherOperationBase getBaseOperation() {
+ return command;
+ }
+
+ public ContainerLaunchRequest getLaunchRequest() {
+ Preconditions.checkState(opType == OPType.LAUNCH_REQUEST);
+ return (ContainerLaunchRequest) command;
+ }
+
+ public ContainerStopRequest getStopRequest() {
+ Preconditions.checkState(opType == OPType.STOP_REQUEST);
+ return (ContainerStopRequest) command;
+ }
+
+ @Override
+ public String toString() {
+ return "ContainerOp{" +
+ "opType=" + opType +
+ ", command=" + command +
+ '}';
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index fe23409..a1b8e29 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -44,11 +44,15 @@ import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
+import org.apache.tez.serviceplugins.api.ContainerLauncher;
+import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
+import org.apache.tez.serviceplugins.api.ContainerStopRequest;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
@@ -60,17 +64,6 @@ import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.TaskAttemptListener;
import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
-import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
-import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
-import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
-import org.apache.tez.dag.app.rm.container.AMContainerEventLaunchFailed;
-import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
-import org.apache.tez.dag.app.rm.container.AMContainerEventType;
-import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
-import org.apache.tez.dag.records.TaskAttemptTerminationCause;
import org.apache.tez.runtime.api.ExecutionContext;
import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
@@ -82,17 +75,17 @@ import org.apache.tez.runtime.task.TezChild;
* Since all (sub)tasks share the same local directory, they must be executed
* sequentially in order to avoid creating/deleting the same files/dirs.
*/
-public class LocalContainerLauncher extends AbstractService implements
- ContainerLauncher {
+public class LocalContainerLauncher extends ContainerLauncher {
private static final Logger LOG = LoggerFactory.getLogger(LocalContainerLauncher.class);
+
private final AppContext context;
private final AtomicBoolean serviceStopped = new AtomicBoolean(false);
private final String workingDirectory;
private final TaskAttemptListener tal;
private final Map<String, String> localEnv;
private final ExecutionContext executionContext;
- private int numExecutors;
+ private final int numExecutors;
private final boolean isPureLocalMode;
private final ConcurrentHashMap<ContainerId, RunningTaskCallback>
@@ -102,23 +95,25 @@ public class LocalContainerLauncher extends AbstractService implements
private final ExecutorService callbackExecutor = Executors.newFixedThreadPool(1,
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("CallbackExecutor").build());
- private BlockingQueue<NMCommunicatorEvent> eventQueue =
- new LinkedBlockingQueue<NMCommunicatorEvent>();
+ private BlockingQueue<ContainerOp> eventQueue = new LinkedBlockingQueue<>();
private Thread eventHandlingThread;
private ListeningExecutorService taskExecutorService;
-
- public LocalContainerLauncher(AppContext context,
+ public LocalContainerLauncher(ContainerLauncherContext containerLauncherContext,
+ AppContext context,
TaskAttemptListener taskAttemptListener,
String workingDirectory,
boolean isPureLocalMode) throws UnknownHostException {
- super(LocalContainerLauncher.class.getName());
+ // TODO Post TEZ-2003. Most of this information is dynamic and only available after the AM
+ // starts up. It's not possible to set these up via a static payload.
+ // Will need some kind of mechanism to dynamically crate payloads / bind to parameters
+ // after the AM starts up.
+ super(LocalContainerLauncher.class.getName(), containerLauncherContext);
this.context = context;
this.tal = taskAttemptListener;
-
this.workingDirectory = workingDirectory;
this.isPureLocalMode = isPureLocalMode;
if (isPureLocalMode) {
@@ -133,11 +128,8 @@ public class LocalContainerLauncher extends AbstractService implements
String host = isPureLocalMode ? InetAddress.getLocalHost().getHostName() :
System.getenv(Environment.NM_HOST.name());
executionContext = new ExecutionContextImpl(host);
- }
- @Override
- public synchronized void serviceInit(Configuration conf) {
- numExecutors = conf.getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
+ numExecutors = getContext().getInitialConfiguration().getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS_DEFAULT);
Preconditions.checkState(numExecutors >=1, "Must have at least 1 executor");
ExecutorService rawExecutor = Executors.newFixedThreadPool(numExecutors,
@@ -169,20 +161,22 @@ public class LocalContainerLauncher extends AbstractService implements
callbackExecutor.shutdownNow();
}
+
+
// Thread to monitor the queue of incoming NMCommunicator events
private class TezSubTaskRunner implements Runnable {
@Override
public void run() {
while (!Thread.currentThread().isInterrupted() && !serviceStopped.get()) {
- NMCommunicatorEvent event;
+ ContainerOp event;
try {
event = eventQueue.take();
- switch (event.getType()) {
- case CONTAINER_LAUNCH_REQUEST:
- launch((NMCommunicatorLaunchRequestEvent) event);
+ switch (event.getOpType()) {
+ case LAUNCH_REQUEST:
+ launch(event.getLaunchRequest());
break;
- case CONTAINER_STOP_REQUEST:
- stop((NMCommunicatorStopRequestEvent)event);
+ case STOP_REQUEST:
+ stop(event.getStopRequest());
break;
}
} catch (InterruptedException e) {
@@ -200,7 +194,7 @@ public class LocalContainerLauncher extends AbstractService implements
@SuppressWarnings("unchecked")
void sendContainerLaunchFailedMsg(ContainerId containerId, String message) {
- context.getEventHandler().handle(new AMContainerEventLaunchFailed(containerId, message));
+ getContext().containerLaunchFailed(containerId, message);
}
private void handleLaunchFailed(Throwable t, ContainerId containerId) {
@@ -215,16 +209,17 @@ public class LocalContainerLauncher extends AbstractService implements
}
//launch tasks
- private void launch(NMCommunicatorLaunchRequestEvent event) {
+ private void launch(ContainerLaunchRequest event) {
String tokenIdentifier = context.getApplicationID().toString();
try {
TezChild tezChild;
try {
+ int taskCommId = context.getTaskCommunicatorIdentifier(event.getTaskCommunicatorName());
tezChild =
createTezChild(context.getAMConf(), event.getContainerId(), tokenIdentifier,
context.getApplicationAttemptId().getAttemptId(), context.getLocalDirs(),
- ((TezTaskCommunicatorImpl)tal.getTaskCommunicator(event.getTaskCommId())).getUmbilical(),
+ ((TezTaskCommunicatorImpl)tal.getTaskCommunicator(taskCommId)).getUmbilical(),
TezCommonUtils.parseCredentialsBytes(event.getContainerLaunchContext().getTokens().array()));
} catch (InterruptedException e) {
handleLaunchFailed(e, event.getContainerId());
@@ -238,7 +233,7 @@ public class LocalContainerLauncher extends AbstractService implements
}
ListenableFuture<TezChild.ContainerExecutionResult> runningTaskFuture =
taskExecutorService.submit(createSubTask(tezChild, event.getContainerId()));
- RunningTaskCallback callback = new RunningTaskCallback(context, event.getContainerId());
+ RunningTaskCallback callback = new RunningTaskCallback(event.getContainerId());
runningContainers.put(event.getContainerId(), callback);
Futures.addCallback(runningTaskFuture, callback, callbackExecutor);
} catch (RejectedExecutionException e) {
@@ -246,7 +241,7 @@ public class LocalContainerLauncher extends AbstractService implements
}
}
- private void stop(NMCommunicatorStopRequestEvent event) {
+ private void stop(ContainerStopRequest event) {
// A stop_request will come in when a task completes and reports back or a preemption decision
// is made. Currently the LocalTaskScheduler does not support preemption. Also preemption
// will not work in local mode till Tez supports task preemption instead of container preemption.
@@ -263,18 +258,15 @@ public class LocalContainerLauncher extends AbstractService implements
// This will need to be fixed once interrupting tasks is supported.
}
// Send this event to maintain regular control flow. This isn't of much use though.
- context.getEventHandler().handle(
- new AMContainerEvent(event.getContainerId(), AMContainerEventType.C_NM_STOP_SENT));
+ getContext().containerStopRequested(event.getContainerId());
}
private class RunningTaskCallback
implements FutureCallback<TezChild.ContainerExecutionResult> {
- private final AppContext appContext;
private final ContainerId containerId;
- RunningTaskCallback(AppContext appContext, ContainerId containerId) {
- this.appContext = appContext;
+ RunningTaskCallback(ContainerId containerId) {
this.containerId = containerId;
}
@@ -286,16 +278,16 @@ public class LocalContainerLauncher extends AbstractService implements
result.getExitStatus() ==
TezChild.ContainerExecutionResult.ExitStatus.ASKED_TO_DIE) {
LOG.info("Container: " + containerId + " completed successfully");
- appContext.getEventHandler().handle(
- new AMContainerEventCompleted(containerId, result.getExitStatus().getExitCode(),
- null, TaskAttemptTerminationCause.CONTAINER_EXITED));
+ getContext()
+ .containerCompleted(containerId, result.getExitStatus().getExitCode(), null,
+ TaskAttemptEndReason.CONTAINER_EXITED);
} else {
LOG.info("Container: " + containerId + " completed but with errors");
- appContext.getEventHandler().handle(
- new AMContainerEventCompleted(containerId, result.getExitStatus().getExitCode(),
- result.getErrorMessage() == null ?
- (result.getThrowable() == null ? null : result.getThrowable().getMessage()) :
- result.getErrorMessage(), TaskAttemptTerminationCause.APPLICATION_ERROR));
+ getContext().containerCompleted(
+ containerId, result.getExitStatus().getExitCode(),
+ result.getErrorMessage() == null ?
+ (result.getThrowable() == null ? null : result.getThrowable().getMessage()) :
+ result.getErrorMessage(), TaskAttemptEndReason.APPLICATION_ERROR);
}
}
@@ -307,16 +299,14 @@ public class LocalContainerLauncher extends AbstractService implements
if (!(t instanceof CancellationException)) {
LOG.info("Container: " + containerId + ": Execution Failed: ", t);
// Inform of failure with exit code 1.
- appContext.getEventHandler()
- .handle(new AMContainerEventCompleted(containerId,
- TezChild.ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE.getExitCode(),
- t.getMessage(), TaskAttemptTerminationCause.APPLICATION_ERROR));
+ getContext().containerCompleted(containerId,
+ TezChild.ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE.getExitCode(),
+ t.getMessage(), TaskAttemptEndReason.APPLICATION_ERROR);
} else {
LOG.info("Ignoring CancellationException - triggered by LocalContainerLauncher");
- appContext.getEventHandler()
- .handle(new AMContainerEventCompleted(containerId,
- TezChild.ContainerExecutionResult.ExitStatus.SUCCESS.getExitCode(),
- "CancellationException", TaskAttemptTerminationCause.CONTAINER_EXITED));
+ getContext().containerCompleted(containerId,
+ TezChild.ContainerExecutionResult.ExitStatus.SUCCESS.getExitCode(),
+ "CancellationException", TaskAttemptEndReason.COMMUNICATION_ERROR.CONTAINER_EXITED);
}
}
}
@@ -334,12 +324,7 @@ public class LocalContainerLauncher extends AbstractService implements
// TezTaskRunner needs to be fixed to ensure this.
Thread.interrupted();
// Inform about the launch request now that the container has been allocated a thread to execute in.
- context.getEventHandler().handle(new AMContainerEventLaunched(containerId));
- ContainerLaunchedEvent lEvt =
- new ContainerLaunchedEvent(containerId, context.getClock().getTime(),
- context.getApplicationAttemptId());
-
- context.getHistoryHandler().handle(new DAGHistoryEvent(context.getCurrentDAGID(), lEvt));
+ getContext().containerLaunched(containerId);
return tezChild.run();
}
};
@@ -368,11 +353,19 @@ public class LocalContainerLauncher extends AbstractService implements
}
+ @Override
+ public void launchContainer(ContainerLaunchRequest launchRequest) {
+ try {
+ eventQueue.put(new ContainerOp(ContainerOp.OPType.LAUNCH_REQUEST, launchRequest));
+ } catch (InterruptedException e) {
+ throw new TezUncheckedException(e);
+ }
+ }
@Override
- public void handle(NMCommunicatorEvent event) {
+ public void stopContainer(ContainerStopRequest stopRequest) {
try {
- eventQueue.put(event);
+ eventQueue.put(new ContainerOp(ContainerOp.OPType.STOP_REQUEST, stopRequest));
} catch (InterruptedException e) {
throw new TezUncheckedException(e);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
index a775948..33763e7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
@@ -18,7 +18,7 @@
package org.apache.tez.dag.app.rm;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
import org.apache.tez.dag.app.dag.TaskAttempt;
import org.apache.tez.dag.records.TezTaskAttemptID;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
index a234e07..ef789c5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
index f86894f..dc50c37 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
@@ -29,14 +29,19 @@ public class NMCommunicatorEvent extends AbstractEvent<NMCommunicatorEventType>
private final NodeId nodeId;
private final Token containerToken;
private final int launcherId;
+ private final int schedulerId;
+ private final int taskCommId;
public NMCommunicatorEvent(ContainerId containerId, NodeId nodeId,
- Token containerToken, NMCommunicatorEventType type, int launcherId) {
+ Token containerToken, NMCommunicatorEventType type, int launcherId,
+ int schedulerId, int taskCommId) {
super(type);
this.containerId = containerId;
this.nodeId = nodeId;
this.containerToken = containerToken;
this.launcherId = launcherId;
+ this.schedulerId = schedulerId;
+ this.taskCommId = taskCommId;
}
public ContainerId getContainerId() {
@@ -55,9 +60,18 @@ public class NMCommunicatorEvent extends AbstractEvent<NMCommunicatorEventType>
return launcherId;
}
+ public int getSchedulerId() {
+ return schedulerId;
+ }
+
+ public int getTaskCommId() {
+ return taskCommId;
+ }
+
public String toSrting() {
return super.toString() + " for container " + containerId + ", nodeId: "
- + nodeId + ", launcherId: " + launcherId;
+ + nodeId + ", launcherId: " + launcherId + ", schedulerId=" + schedulerId +
+ ", taskCommId=" + taskCommId;
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
index a38345c..c57b6be 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
@@ -26,15 +26,14 @@ public class NMCommunicatorLaunchRequestEvent extends NMCommunicatorEvent {
private final ContainerLaunchContext clc;
private final Container container;
// The task communicator index for the specific container being launched.
- private final int taskCommId;
public NMCommunicatorLaunchRequestEvent(ContainerLaunchContext clc,
- Container container, int launcherId, int taskCommId) {
+ Container container, int launcherId, int schedulerId, int taskCommId) {
super(container.getId(), container.getNodeId(), container
- .getContainerToken(), NMCommunicatorEventType.CONTAINER_LAUNCH_REQUEST, launcherId);
+ .getContainerToken(), NMCommunicatorEventType.CONTAINER_LAUNCH_REQUEST,
+ launcherId, schedulerId, taskCommId);
this.clc = clc;
this.container = container;
- this.taskCommId = taskCommId;
}
public ContainerLaunchContext getContainerLaunchContext() {
@@ -45,10 +44,6 @@ public class NMCommunicatorLaunchRequestEvent extends NMCommunicatorEvent {
return container;
}
- public int getTaskCommId() {
- return taskCommId;
- }
-
@Override
public boolean equals(Object o) {
if (this == o) {
http://git-wip-us.apache.org/repos/asf/tez/blob/d799d3b9/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
index c9b5c44..352f450 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.yarn.api.records.Token;
public class NMCommunicatorStopRequestEvent extends NMCommunicatorEvent {
public NMCommunicatorStopRequestEvent(ContainerId containerId, NodeId nodeId,
- Token containerToken, int launcherId) {
+ Token containerToken, int launcherId, int schedulerId, int taskCommId) {
super(containerId, nodeId, containerToken,
- NMCommunicatorEventType.CONTAINER_STOP_REQUEST, launcherId);
+ NMCommunicatorEventType.CONTAINER_STOP_REQUEST, launcherId, schedulerId, taskCommId);
}
}
[12/51] [abbrv] tez git commit: TEZ-2131. Add additional tests for
tasks running in the AM. (sseth)
Posted by ss...@apache.org.
TEZ-2131. Add additional tests for tasks running in the AM. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/8b79df52
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/8b79df52
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/8b79df52
Branch: refs/heads/TEZ-2003
Commit: 8b79df5286cd79f3e850de00749fd3d949b97962
Parents: 248a994
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 17:23:18 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/tests/TestExternalTezServices.java | 10 +++++++---
2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/8b79df52/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 76496c9..4b0a12b 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -6,5 +6,6 @@ ALL CHANGES:
TEZ-2122. Setup pluggable components at AM/Vertex level.
TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
+ TEZ-2131. Add additional tests for tasks running in the AM.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/8b79df52/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 01c2080..0ec972b 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -40,7 +40,6 @@ import org.apache.tez.service.MiniTezTestServiceCluster;
import org.apache.tez.test.MiniTezCluster;
import org.junit.AfterClass;
import org.junit.BeforeClass;
-import org.junit.Ignore;
import org.junit.Test;
public class TestExternalTezServices {
@@ -229,6 +228,13 @@ public class TestExternalTezServices {
}
@Test(timeout = 60000)
+ public void testAllInAM() throws Exception {
+ int expectedExternalSubmissions = 0; // All in AM
+ runJoinValidate("AllInAM", expectedExternalSubmissions, PROPS_IN_AM,
+ PROPS_IN_AM, PROPS_IN_AM);
+ }
+
+ @Test(timeout = 60000)
public void testMixed1() throws Exception { // M-ExtService, R-containers
int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 0 for num reducers.
runJoinValidate("Mixed1", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
@@ -263,8 +269,6 @@ public class TestExternalTezServices {
PROPS_EXT_SERVICE_PUSH, PROPS_IN_AM);
}
-
- @Ignore // Re-activate this after the AM registers the shuffle token with the launcher.
@Test(timeout = 60000)
public void testMixed6() throws Exception { // M - AM, R - Service
int expectedExternalSubmissions = 0 + 3; // 3 for R in service
[43/51] [abbrv] tez git commit: TEZ-2652. Cleanup the way services
are specified for an AM and vertices. (sseth)
Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 7c6a6a4..594e6d3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -17,18 +17,21 @@ package org.apache.tez.dag.app.launcher;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.net.UnknownHostException;
+import java.util.List;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
import org.apache.tez.serviceplugins.api.ContainerStopRequest;
-import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerLauncherContextImpl;
@@ -63,35 +66,35 @@ public class ContainerLauncherRouter extends AbstractService
public ContainerLauncherRouter(Configuration conf, AppContext context,
TaskAttemptListener taskAttemptListener,
String workingDirectory,
- String[] containerLauncherClassIdentifiers,
+ List<NamedEntityDescriptor> containerLauncherDescriptors,
boolean isPureLocalMode) throws UnknownHostException {
super(ContainerLauncherRouter.class.getName());
this.appContext = context;
- if (containerLauncherClassIdentifiers == null || containerLauncherClassIdentifiers.length == 0) {
+ if (containerLauncherDescriptors == null || containerLauncherDescriptors.isEmpty()) {
if (isPureLocalMode) {
- containerLauncherClassIdentifiers =
- new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+ containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
+ TezConstants.getTezUberServicePluginName(), null));
} else {
- containerLauncherClassIdentifiers =
- new String[]{TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
+ TezConstants.getTezYarnServicePluginName(), null));
}
}
- containerLauncherContexts = new ContainerLauncherContext[containerLauncherClassIdentifiers.length];
- containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
- containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[containerLauncherClassIdentifiers.length];
+ containerLauncherContexts = new ContainerLauncherContext[containerLauncherDescriptors.size()];
+ containerLaunchers = new ContainerLauncher[containerLauncherDescriptors.size()];
+ containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[containerLauncherDescriptors.size()];
- for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
+ for (int i = 0; i < containerLauncherDescriptors.size(); i++) {
ContainerLauncherContext containerLauncherContext = new ContainerLauncherContextImpl(context, taskAttemptListener);
containerLauncherContexts[i] = containerLauncherContext;
- containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
+ containerLaunchers[i] = createContainerLauncher(containerLauncherDescriptors.get(i), context,
containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode, conf);
containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
}
}
- private ContainerLauncher createContainerLauncher(String containerLauncherClassIdentifier,
+ private ContainerLauncher createContainerLauncher(NamedEntityDescriptor containerLauncherDescriptor,
AppContext context,
ContainerLauncherContext containerLauncherContext,
TaskAttemptListener taskAttemptListener,
@@ -99,11 +102,12 @@ public class ContainerLauncherRouter extends AbstractService
boolean isPureLocalMode,
Configuration conf) throws
UnknownHostException {
- if (containerLauncherClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+ if (containerLauncherDescriptor.getEntityName().equals(
+ TezConstants.getTezYarnServicePluginName())) {
LOG.info("Creating DefaultContainerLauncher");
return new ContainerLauncherImpl(containerLauncherContext);
- } else if (containerLauncherClassIdentifier
- .equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+ } else if (containerLauncherDescriptor.getEntityName()
+ .equals(TezConstants.getTezUberServicePluginName())) {
LOG.info("Creating LocalContainerLauncher");
// TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
// extensive internals which are only available at runtime. Will likely require
@@ -111,10 +115,10 @@ public class ContainerLauncherRouter extends AbstractService
return
new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener, workingDirectory, isPureLocalMode);
} else {
- LOG.info("Creating container launcher : " + containerLauncherClassIdentifier);
+ LOG.info("Creating container launcher {}:{} ", containerLauncherDescriptor.getEntityName(), containerLauncherDescriptor.getClassName());
Class<? extends ContainerLauncher> containerLauncherClazz =
(Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
- containerLauncherClassIdentifier);
+ containerLauncherDescriptor.getClassName());
try {
Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
.getConstructor(ContainerLauncherContext.class);
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index d8cf080..67a088e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -22,6 +22,7 @@ import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
+import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
@@ -34,6 +35,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.serviceplugins.api.TaskScheduler;
import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
@@ -55,7 +58,6 @@ import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.TaskLocationHint.TaskBasedLocationAffinity;
@@ -143,14 +145,14 @@ public class TaskSchedulerEventHandler extends AbstractService implements
* @param eventHandler
* @param containerSignatureMatcher
* @param webUI
- * @param schedulerClasses the list of scheduler classes / codes. Tez internal classes are represented as codes.
+ * @param schedulerDescriptors the list of scheduler descriptors. Tez internal classes will not have the class names populated.
* An empty list defaults to using the YarnTaskScheduler as the only source.
*/
@SuppressWarnings("rawtypes")
public TaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
- String [] schedulerClasses, boolean isPureLocalMode) {
+ List<NamedEntityDescriptor> schedulerDescriptors, boolean isPureLocalMode) {
super(TaskSchedulerEventHandler.class.getName());
this.appContext = appContext;
this.eventHandler = eventHandler;
@@ -166,31 +168,34 @@ public class TaskSchedulerEventHandler extends AbstractService implements
// Override everything for pure local mode
if (isPureLocalMode) {
- this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+ this.taskSchedulerClasses = new String[] {TezConstants.getTezUberServicePluginName()};
this.yarnTaskSchedulerIndex = -1;
} else {
- if (schedulerClasses == null || schedulerClasses.length ==0) {
- this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ if (schedulerDescriptors == null || schedulerDescriptors.isEmpty()) {
+ this.taskSchedulerClasses = new String[] {TezConstants.getTezYarnServicePluginName()};
this.yarnTaskSchedulerIndex = 0;
} else {
// Ensure the YarnScheduler will be setup and note it's index. This will be responsible for heartbeats and YARN registration.
int foundYarnTaskSchedulerIndex = -1;
- for (int i = 0 ; i < schedulerClasses.length ; i++) {
- if (schedulerClasses[i].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+
+ List<String> taskSchedulerClassList = new LinkedList<>();
+ for (int i = 0 ; i < schedulerDescriptors.size() ; i++) {
+ if (schedulerDescriptors.get(i).getEntityName().equals(
+ TezConstants.getTezYarnServicePluginName())) {
+ taskSchedulerClassList.add(schedulerDescriptors.get(i).getEntityName());
foundYarnTaskSchedulerIndex = i;
- break;
+ } else if (schedulerDescriptors.get(i).getEntityName().equals(
+ TezConstants.getTezUberServicePluginName())) {
+ taskSchedulerClassList.add(schedulerDescriptors.get(i).getEntityName());
+ } else {
+ taskSchedulerClassList.add(schedulerDescriptors.get(i).getClassName());
}
}
- if (foundYarnTaskSchedulerIndex == -1) { // Not found. Add at the end.
- this.taskSchedulerClasses = new String[schedulerClasses.length+1];
- foundYarnTaskSchedulerIndex = this.taskSchedulerClasses.length -1;
- for (int i = 0 ; i < schedulerClasses.length ; i++) { // Copy over the rest.
- this.taskSchedulerClasses[i] = schedulerClasses[i];
- }
- this.taskSchedulerClasses[foundYarnTaskSchedulerIndex] = TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT;
- } else {
- this.taskSchedulerClasses = schedulerClasses;
+ if (foundYarnTaskSchedulerIndex == -1) {
+ taskSchedulerClassList.add(YarnTaskSchedulerService.class.getName());
+ foundYarnTaskSchedulerIndex = taskSchedulerClassList.size() -1;
}
+ this.taskSchedulerClasses = taskSchedulerClassList.toArray(new String[taskSchedulerClassList.size()]);
this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
}
}
@@ -419,10 +424,10 @@ public class TaskSchedulerEventHandler extends AbstractService implements
new TaskSchedulerContextImpl(this, appContext, schedulerId, trackingUrl,
customAppIdIdentifier, host, port, getConfig());
TaskSchedulerContext wrappedContext = new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
- if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+ if (schedulerClassName.equals(TezConstants.getTezYarnServicePluginName())) {
LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
return new YarnTaskSchedulerService(wrappedContext);
- } else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
+ } else if (schedulerClassName.equals(TezConstants.getTezUberServicePluginName())) {
LOG.info("Creating TaskScheduler: Local TaskScheduler");
return new LocalTaskSchedulerService(wrappedContext);
} else {
@@ -454,7 +459,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
for (int i = 0; i < taskSchedulerClasses.length; i++) {
long customAppIdIdentifier;
if (isPureLocalMode || taskSchedulerClasses[i].equals(
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) { // Use the app identifier from the appId.
+ TezConstants.getTezYarnServicePluginName())) { // Use the app identifier from the appId.
customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp();
} else {
customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT);
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 21ae5f7..17feeaa 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -486,7 +487,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
Credentials credentials, String jobUserName, int handlerConcurrency, int numConcurrentContainers) {
super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
isSession, workingDirectory, localDirs, logDirs, new TezApiVersionInfo().getVersion(), 1,
- credentials, jobUserName);
+ credentials, jobUserName, null);
containerLauncherContext = new ContainerLauncherContextImpl(getContext(), getTaskAttemptListener());
containerLauncher = new MockContainerLauncher(launcherGoFlag, containerLauncherContext);
shutdownHandler = new MockDAGAppMasterShutdownHandler();
@@ -500,7 +501,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
// use mock container launcher for tests
@Override
protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
- String[] containerLaunchers,
+ List<NamedEntityDescriptor> containerLauncherDescirptors,
boolean isLocal)
throws UnknownHostException {
return new ContainerLauncherRouter(containerLauncher, getContext());
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 41a7373..e45b0a2 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -50,6 +50,7 @@ import org.apache.tez.common.ContainerTask;
import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
@@ -375,10 +376,10 @@ public class TestTaskAttemptListenerImplTezDag {
public TaskAttemptListenerImplForTest(AppContext context,
TaskHeartbeatHandler thh,
ContainerHeartbeatHandler chh,
- String[] taskCommunicatorClassIdentifiers,
+ List<NamedEntityDescriptor> taskCommDescriptors,
Configuration conf,
boolean isPureLocalMode) {
- super(context, thh, chh, taskCommunicatorClassIdentifiers, conf,
+ super(context, thh, chh, taskCommDescriptors, conf,
isPureLocalMode);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index 3ea0446..f191175 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -32,6 +32,7 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -48,6 +49,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.client.DAGClientServer;
@@ -91,7 +93,7 @@ public class TestTaskSchedulerEventHandler {
public MockTaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
- super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new String[] {}, false);
+ super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new LinkedList<NamedEntityDescriptor>(), false);
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index 966c95a..60d37e9 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
@@ -130,7 +131,7 @@ class TestTaskSchedulerHelpers {
EventHandler eventHandler,
TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
ContainerSignatureMatcher containerSignatureMatcher) {
- super(appContext, null, eventHandler, containerSignatureMatcher, null, new String[]{}, false);
+ super(appContext, null, eventHandler, containerSignatureMatcher, null, new LinkedList<NamedEntityDescriptor>(), false);
this.amrmClientAsync = amrmClientAsync;
this.containerSignatureMatcher = containerSignatureMatcher;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
index ba17ba0..611e8cc 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
@@ -20,8 +20,8 @@ package org.apache.tez.examples;
import java.io.IOException;
import java.util.Set;
-import java.util.Map;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -137,6 +137,9 @@ public class JoinValidate extends TezExampleBase {
private DAG createDag(TezConfiguration tezConf, Path lhs, Path rhs, int numPartitions)
throws IOException {
DAG dag = DAG.create(getDagName());
+ if (getDefaultExecutionContext() != null) {
+ dag.setExecutionContext(getDefaultExecutionContext());
+ }
// Configuration for intermediate output - shared by Vertex1 and Vertex2
// This should only be setting selective keys from the underlying conf. Fix after there's a
@@ -153,18 +156,18 @@ public class JoinValidate extends TezExampleBase {
MRInput
.createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
lhs.toUri().toString()).groupSplits(!isDisableSplitGrouping()).build());
- setVertexProperties(lhsVertex, getLhsVertexProperties());
+ setVertexExecutionContext(lhsVertex, getLhsExecutionContext());
Vertex rhsVertex = Vertex.create(RHS_INPUT_NAME, ProcessorDescriptor.create(
ForwardingProcessor.class.getName())).addDataSource("rhs",
MRInput
.createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
rhs.toUri().toString()).groupSplits(!isDisableSplitGrouping()).build());
- setVertexProperties(rhsVertex, getRhsVertexProperties());
+ setVertexExecutionContext(rhsVertex, getRhsExecutionContext());
Vertex joinValidateVertex = Vertex.create("joinvalidate", ProcessorDescriptor.create(
JoinValidateProcessor.class.getName()), numPartitions);
- setVertexProperties(joinValidateVertex, getValidateVertexProperties());
+ setVertexExecutionContext(joinValidateVertex, getValidateExecutionContext());
Edge e1 = Edge.create(lhsVertex, joinValidateVertex, edgeConf.createDefaultEdgeProperty());
Edge e2 = Edge.create(rhsVertex, joinValidateVertex, edgeConf.createDefaultEdgeProperty());
@@ -174,23 +177,25 @@ public class JoinValidate extends TezExampleBase {
return dag;
}
- private void setVertexProperties(Vertex vertex, Map<String, String> properties) {
- if (properties != null) {
- for (Map.Entry<String, String> entry : properties.entrySet()) {
- vertex.setConf(entry.getKey(), entry.getValue());
- }
+ private void setVertexExecutionContext(Vertex vertex, VertexExecutionContext executionContext) {
+ if (executionContext != null) {
+ vertex.setExecutionContext(executionContext);
}
}
- protected Map<String, String> getLhsVertexProperties() {
+ protected VertexExecutionContext getDefaultExecutionContext() {
return null;
}
- protected Map<String, String> getRhsVertexProperties() {
+ protected VertexExecutionContext getLhsExecutionContext() {
return null;
}
- protected Map<String, String> getValidateVertexProperties() {
+ protected VertexExecutionContext getRhsExecutionContext() {
+ return null;
+ }
+
+ protected VertexExecutionContext getValidateExecutionContext() {
return null;
}
@@ -240,4 +245,6 @@ public class JoinValidate extends TezExampleBase {
}
}
}
+
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index 85f9415..0002b42 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -121,7 +121,8 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
private RunContainerRequestProto constructRunContainerRequest(ContainerLaunchRequest launchRequest) throws
IOException {
RunContainerRequestProto.Builder builder = RunContainerRequestProto.newBuilder();
- Preconditions.checkArgument(launchRequest.getTaskCommunicatorName().equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT));
+ Preconditions.checkArgument(launchRequest.getTaskCommunicatorName().equals(
+ TezConstants.getTezYarnServicePluginName()));
InetSocketAddress address = (InetSocketAddress) getContext().getTaskCommunicatorMetaInfo(launchRequest.getTaskCommunicatorName());
builder.setAmHost(address.getHostName()).setAmPort(address.getPort());
builder.setAppAttemptNumber(appAttemptId.getAttemptId());
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java b/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
index e5d2e3b..f31476f 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/examples/JoinValidateConfigured.java
@@ -14,36 +14,46 @@
package org.apache.tez.examples;
-import java.util.Map;
+
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
public class JoinValidateConfigured extends JoinValidate {
- private final Map<String, String> lhsProps;
- private final Map<String, String> rhsProps;
- private final Map<String, String> validateProps;
+ private final VertexExecutionContext defaultExecutionContext;
+ private final VertexExecutionContext lhsContext;
+ private final VertexExecutionContext rhsContext;
+ private final VertexExecutionContext validateContext;
private final String dagNameSuffix;
- public JoinValidateConfigured(Map<String, String> lhsProps, Map<String, String> rhsProps,
- Map<String, String> validateProps, String dagNameSuffix) {
- this.lhsProps = lhsProps;
- this.rhsProps = rhsProps;
- this.validateProps = validateProps;
+ public JoinValidateConfigured(VertexExecutionContext defaultExecutionContext,
+ VertexExecutionContext lhsContext,
+ VertexExecutionContext rhsContext,
+ VertexExecutionContext validateContext, String dagNameSuffix) {
+ this.defaultExecutionContext = defaultExecutionContext;
+ this.lhsContext = lhsContext;
+ this.rhsContext = rhsContext;
+ this.validateContext = validateContext;
this.dagNameSuffix = dagNameSuffix;
}
@Override
- protected Map<String, String> getLhsVertexProperties() {
- return this.lhsProps;
+ protected VertexExecutionContext getDefaultExecutionContext() {
+ return this.defaultExecutionContext;
+ }
+
+ @Override
+ protected VertexExecutionContext getLhsExecutionContext() {
+ return this.lhsContext;
}
@Override
- protected Map<String, String> getRhsVertexProperties() {
- return this.rhsProps;
+ protected VertexExecutionContext getRhsExecutionContext() {
+ return this.rhsContext;
}
@Override
- protected Map<String, String> getValidateVertexProperties() {
- return this.validateProps;
+ protected VertexExecutionContext getValidateExecutionContext() {
+ return this.validateContext;
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 45c70f1..07dd363 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.Map;
-import com.google.common.collect.Maps;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -28,9 +27,9 @@ import org.apache.tez.client.TezClient;
import org.apache.tez.dag.api.DAG;
import org.apache.tez.dag.api.ProcessorDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
import org.apache.tez.dag.api.client.DAGClient;
import org.apache.tez.dag.api.client.DAGStatus;
import org.apache.tez.dag.app.launcher.TezTestServiceNoOpContainerLauncher;
@@ -43,6 +42,10 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
import org.apache.tez.runtime.library.processor.SleepProcessor;
import org.apache.tez.service.MiniTezTestServiceCluster;
import org.apache.tez.service.impl.ContainerRunnerImpl;
+import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor;
+import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
+import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
+import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
import org.apache.tez.test.MiniTezCluster;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@@ -72,9 +75,15 @@ public class TestExternalTezServices {
private static final Path HASH_JOIN_EXPECTED_RESULT_PATH = new Path(SRC_DATA_DIR, "expectedOutputPath");
private static final Path HASH_JOIN_OUTPUT_PATH = new Path(SRC_DATA_DIR, "outPath");
- private static final Map<String, String> PROPS_EXT_SERVICE_PUSH = Maps.newHashMap();
- private static final Map<String, String> PROPS_REGULAR_CONTAINERS = Maps.newHashMap();
- private static final Map<String, String> PROPS_IN_AM = Maps.newHashMap();
+ private static final VertexExecutionContext EXECUTION_CONTEXT_EXT_SERVICE_PUSH =
+ VertexExecutionContext.create(
+ EXT_PUSH_ENTITY_NAME, EXT_PUSH_ENTITY_NAME, EXT_PUSH_ENTITY_NAME);
+ private static final VertexExecutionContext EXECUTION_CONTEXT_REGULAR_CONTAINERS =
+ VertexExecutionContext.createExecuteInContainers(true);
+ private static final VertexExecutionContext EXECUTION_CONTEXT_IN_AM =
+ VertexExecutionContext.createExecuteInAm(true);
+
+ private static final VertexExecutionContext EXECUTION_CONTEXT_DEFAULT = EXECUTION_CONTEXT_EXT_SERVICE_PUSH;
private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestExternalTezServices.class.getName()
+ "-tmpDir";
@@ -127,51 +136,28 @@ public class TestExternalTezServices {
confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
confForJobs.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false);
- confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_SCHEDULERS,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
- EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskSchedulerService.class.getName());
-
- confForJobs.setStrings(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHERS,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
- EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceNoOpContainerLauncher.class.getName());
-
- confForJobs.setStrings(TezConfiguration.TEZ_AM_TASK_COMMUNICATORS,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT,
- EXT_PUSH_ENTITY_NAME + ":" + TezTestServiceTaskCommunicatorImpl.class.getName());
-
- // Default all jobs to run via the service. Individual tests override this on a per vertex/dag level.
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
- confForJobs.set(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
-
- // Setup various executor sets
- PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
- PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
- PROPS_REGULAR_CONTAINERS.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT);
-
- PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME, EXT_PUSH_ENTITY_NAME);
- PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME, EXT_PUSH_ENTITY_NAME);
- PROPS_EXT_SERVICE_PUSH.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME, EXT_PUSH_ENTITY_NAME);
-
- PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_TASK_SCHEDULER_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
- PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_CONTAINER_LAUNCHER_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
- PROPS_IN_AM.put(TezConfiguration.TEZ_AM_VERTEX_TASK_COMMUNICATOR_NAME,
- TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT);
+ TaskSchedulerDescriptor[] taskSchedulerDescriptors = new TaskSchedulerDescriptor[]{
+ TaskSchedulerDescriptor
+ .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskSchedulerService.class.getName())};
+
+ ContainerLauncherDescriptor[] containerLauncherDescriptors = new ContainerLauncherDescriptor[]{
+ ContainerLauncherDescriptor
+ .create(EXT_PUSH_ENTITY_NAME, TezTestServiceNoOpContainerLauncher.class.getName())};
+
+ TaskCommunicatorDescriptor[] taskCommunicatorDescriptors = new TaskCommunicatorDescriptor[]{
+ TaskCommunicatorDescriptor
+ .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskCommunicatorImpl.class.getName())};
+ ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor.create(true, true,
+ taskSchedulerDescriptors, containerLauncherDescriptors, taskCommunicatorDescriptors);
// Create a session to use for all tests.
TezConfiguration tezClientConf = new TezConfiguration(confForJobs);
- sharedTezClient = TezClient.create(TestExternalTezServices.class.getSimpleName() + "_session",
- tezClientConf, true);
+ sharedTezClient = TezClient
+ .newBuilder(TestExternalTezServices.class.getSimpleName() + "_session", tezClientConf)
+ .setIsSession(true).setServicePluginDescriptor(servicePluginsDescriptor).build();
+
sharedTezClient.start();
LOG.info("Shared TezSession started");
sharedTezClient.waitTillReady();
@@ -225,71 +211,71 @@ public class TestExternalTezServices {
@Test(timeout = 60000)
public void testAllInService() throws Exception {
int expectedExternalSubmissions = 4 + 3; //4 for 4 src files, 3 for num reducers.
- runJoinValidate("AllInService", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
- PROPS_EXT_SERVICE_PUSH, PROPS_EXT_SERVICE_PUSH);
+ runJoinValidate("AllInService", expectedExternalSubmissions, EXECUTION_CONTEXT_EXT_SERVICE_PUSH,
+ EXECUTION_CONTEXT_EXT_SERVICE_PUSH, EXECUTION_CONTEXT_EXT_SERVICE_PUSH);
}
@Test(timeout = 60000)
public void testAllInContainers() throws Exception {
int expectedExternalSubmissions = 0; // All in containers
- runJoinValidate("AllInContainers", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
- PROPS_REGULAR_CONTAINERS, PROPS_REGULAR_CONTAINERS);
+ runJoinValidate("AllInContainers", expectedExternalSubmissions, EXECUTION_CONTEXT_REGULAR_CONTAINERS,
+ EXECUTION_CONTEXT_REGULAR_CONTAINERS, EXECUTION_CONTEXT_REGULAR_CONTAINERS);
}
@Test(timeout = 60000)
public void testAllInAM() throws Exception {
int expectedExternalSubmissions = 0; // All in AM
- runJoinValidate("AllInAM", expectedExternalSubmissions, PROPS_IN_AM,
- PROPS_IN_AM, PROPS_IN_AM);
+ runJoinValidate("AllInAM", expectedExternalSubmissions, EXECUTION_CONTEXT_IN_AM,
+ EXECUTION_CONTEXT_IN_AM, EXECUTION_CONTEXT_IN_AM);
}
@Test(timeout = 60000)
public void testMixed1() throws Exception { // M-ExtService, R-containers
int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 0 for num reducers.
- runJoinValidate("Mixed1", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
- PROPS_EXT_SERVICE_PUSH, PROPS_REGULAR_CONTAINERS);
+ runJoinValidate("Mixed1", expectedExternalSubmissions, EXECUTION_CONTEXT_EXT_SERVICE_PUSH,
+ EXECUTION_CONTEXT_EXT_SERVICE_PUSH, EXECUTION_CONTEXT_REGULAR_CONTAINERS);
}
@Test(timeout = 60000)
public void testMixed2() throws Exception { // M-Containers, R-ExtService
int expectedExternalSubmissions = 0 + 3; // 3 for num reducers.
- runJoinValidate("Mixed2", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
- PROPS_REGULAR_CONTAINERS, PROPS_EXT_SERVICE_PUSH);
+ runJoinValidate("Mixed2", expectedExternalSubmissions, EXECUTION_CONTEXT_REGULAR_CONTAINERS,
+ EXECUTION_CONTEXT_REGULAR_CONTAINERS, EXECUTION_CONTEXT_EXT_SERVICE_PUSH);
}
@Test(timeout = 60000)
public void testMixed3() throws Exception { // M - service, R-AM
int expectedExternalSubmissions = 4 + 0; //4 for 4 src files, 0 for num reducers (in-AM).
- runJoinValidate("Mixed3", expectedExternalSubmissions, PROPS_EXT_SERVICE_PUSH,
- PROPS_EXT_SERVICE_PUSH, PROPS_IN_AM);
+ runJoinValidate("Mixed3", expectedExternalSubmissions, EXECUTION_CONTEXT_EXT_SERVICE_PUSH,
+ EXECUTION_CONTEXT_EXT_SERVICE_PUSH, EXECUTION_CONTEXT_IN_AM);
}
@Test(timeout = 60000)
public void testMixed4() throws Exception { // M - containers, R-AM
int expectedExternalSubmissions = 0 + 0; // Nothing in external service.
- runJoinValidate("Mixed4", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
- PROPS_REGULAR_CONTAINERS, PROPS_IN_AM);
+ runJoinValidate("Mixed4", expectedExternalSubmissions, EXECUTION_CONTEXT_REGULAR_CONTAINERS,
+ EXECUTION_CONTEXT_REGULAR_CONTAINERS, EXECUTION_CONTEXT_IN_AM);
}
@Test(timeout = 60000)
public void testMixed5() throws Exception { // M1 - containers, M2-extservice, R-AM
int expectedExternalSubmissions = 2 + 0; // 2 for M2
- runJoinValidate("Mixed5", expectedExternalSubmissions, PROPS_REGULAR_CONTAINERS,
- PROPS_EXT_SERVICE_PUSH, PROPS_IN_AM);
+ runJoinValidate("Mixed5", expectedExternalSubmissions, EXECUTION_CONTEXT_REGULAR_CONTAINERS,
+ EXECUTION_CONTEXT_EXT_SERVICE_PUSH, EXECUTION_CONTEXT_IN_AM);
}
@Test(timeout = 60000)
public void testMixed6() throws Exception { // M - AM, R - Service
int expectedExternalSubmissions = 0 + 3; // 3 for R in service
- runJoinValidate("Mixed6", expectedExternalSubmissions, PROPS_IN_AM,
- PROPS_IN_AM, PROPS_EXT_SERVICE_PUSH);
+ runJoinValidate("Mixed6", expectedExternalSubmissions, EXECUTION_CONTEXT_IN_AM,
+ EXECUTION_CONTEXT_IN_AM, EXECUTION_CONTEXT_EXT_SERVICE_PUSH);
}
@Test(timeout = 60000)
public void testMixed7() throws Exception { // M - AM, R - Containers
int expectedExternalSubmissions = 0; // Nothing in ext service
- runJoinValidate("Mixed7", expectedExternalSubmissions, PROPS_IN_AM,
- PROPS_IN_AM, PROPS_REGULAR_CONTAINERS);
+ runJoinValidate("Mixed7", expectedExternalSubmissions, EXECUTION_CONTEXT_IN_AM,
+ EXECUTION_CONTEXT_IN_AM, EXECUTION_CONTEXT_REGULAR_CONTAINERS);
}
@Test(timeout = 60000)
@@ -303,10 +289,9 @@ public class TestExternalTezServices {
DAG dag = DAG.create(ContainerRunnerImpl.DAG_NAME_INSTRUMENTED_FAILURES);
Vertex v =Vertex.create("Vertex1", ProcessorDescriptor.create(SleepProcessor.class.getName()),
3);
- for (Map.Entry<String, String> prop : PROPS_EXT_SERVICE_PUSH.entrySet()) {
- v.setConf(prop.getKey(), prop.getValue());
- }
+ v.setExecutionContext(EXECUTION_CONTEXT_EXT_SERVICE_PUSH);
dag.addVertex(v);
+
DAGClient dagClient = sharedTezClient.submitDAG(dag);
DAGStatus dagStatus = dagClient.waitForCompletion();
assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState());
@@ -315,16 +300,16 @@ public class TestExternalTezServices {
}
- private void runJoinValidate(String name, int extExpectedCount, Map<String, String> lhsProps,
- Map<String, String> rhsProps,
- Map<String, String> validateProps) throws
+ private void runJoinValidate(String name, int extExpectedCount, VertexExecutionContext lhsContext,
+ VertexExecutionContext rhsContext,
+ VertexExecutionContext validateContext) throws
Exception {
int externalSubmissionCount = tezTestServiceCluster.getNumSubmissions();
TezConfiguration tezConf = new TezConfiguration(confForJobs);
JoinValidateConfigured joinValidate =
- new JoinValidateConfigured(lhsProps, rhsProps,
- validateProps, name);
+ new JoinValidateConfigured(EXECUTION_CONTEXT_DEFAULT, lhsContext, rhsContext,
+ validateContext, name);
String[] validateArgs = new String[]{"-disableSplitGrouping",
HASH_JOIN_EXPECTED_RESULT_PATH.toString(), HASH_JOIN_OUTPUT_PATH.toString(), "3"};
assertEquals(0, joinValidate.run(tezConf, validateArgs, sharedTezClient));
http://git-wip-us.apache.org/repos/asf/tez/blob/5afbef1b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index fff39a0..353fe23 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -62,6 +62,7 @@ import org.apache.tez.common.security.TokenCache;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.records.DAGProtos;
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.dag.utils.RelocalizationUtils;
import org.apache.tez.runtime.api.ExecutionContext;
@@ -477,7 +478,9 @@ public class TezChild {
}
// Security framework already loaded the tokens into current ugi
- TezUtilsInternal.addUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()), defaultConf);
+ DAGProtos.ConfigurationProto confProto =
+ TezUtilsInternal.readUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name()));
+ TezUtilsInternal.addUserSpecifiedTezConfiguration(defaultConf, confProto.getConfKeyValuesList());
UserGroupInformation.setConfiguration(defaultConf);
Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
TezChild tezChild = newTezChild(defaultConf, host, port, containerIdentifier,
[51/51] [abbrv] tez git commit: TEZ-2698. rebase 08/05 (sseth)
Posted by ss...@apache.org.
TEZ-2698. rebase 08/05 (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/5fdbe04f
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/5fdbe04f
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/5fdbe04f
Branch: refs/heads/TEZ-2003
Commit: 5fdbe04ffbd1cb7fb9809ee8446dc7e523ba35a9
Parents: 4a21040
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Aug 6 02:20:32 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 02:20:32 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../test/java/org/apache/tez/client/TestTezClientUtils.java | 2 +-
.../org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java | 1 +
.../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java | 5 ++++-
.../src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java | 6 +++---
.../apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java | 4 +---
.../apache/tez/runtime/library/common/shuffle/TestFetcher.java | 2 +-
7 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/5fdbe04f/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index f921739..b133ea3 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -43,5 +43,6 @@ ALL CHANGES:
TEZ-2657. Add tests for client side changes - specifying plugins, etc.
TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs.
TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators.
+ TEZ-2698. rebase 08/05
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/5fdbe04f/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
index 8f40bbd..d1033b2 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
@@ -225,7 +225,7 @@ public class TestTezClientUtils {
appId, null, "dagname",
amConf, m,
credentials, false,
- new TezApiVersionInfo(), null);
+ new TezApiVersionInfo(), null, null);
assertEquals(testpriority, appcontext.getPriority().getPriority());
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5fdbe04f/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 7b97738..6c1dad9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -32,6 +32,7 @@ import org.apache.commons.collections4.ListUtils;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
http://git-wip-us.apache.org/repos/asf/tez/blob/5fdbe04f/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index fb6d5e7..d3f1c44 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -163,7 +163,10 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
}
server.start();
- this.address = NetUtils.getConnectAddress(server);
+ InetSocketAddress serverBindAddress = NetUtils.getConnectAddress(server);
+ this.address = NetUtils.createSocketAddrForHost(
+ serverBindAddress.getAddress().getCanonicalHostName(),
+ serverBindAddress.getPort());
LOG.info("Instantiated TezTaskCommunicator RPC at " + this.address);
} catch (IOException e) {
throw new TezUncheckedException(e);
http://git-wip-us.apache.org/repos/asf/tez/blob/5fdbe04f/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 2e6e568..b04b461 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -401,7 +401,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
List<TezEvent> events = Lists.newArrayListWithCapacity(
cData.taskSpec.getOutputs().size() + 1);
if (cData.numUpdates == 0 && eventsDelegate != null) {
- eventsDelegate.getEvents(cData.taskSpec, events, getContext().getClock().getTime());
+ eventsDelegate.getEvents(cData.taskSpec, events, MockDAGAppMaster.this.getContext().getClock().getTime());
}
TezCounters counters = null;
if (countersDelegate != null) {
@@ -416,7 +416,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
float progress = updateProgress ? cData.numUpdates/maxUpdates : 0f;
events.add(new TezEvent(new TaskStatusUpdateEvent(counters, progress, stats), new EventMetaData(
EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId),
- getContext().getClock().getTime()));
+ MockDAGAppMaster.this.getContext().getClock().getTime()));
// TezHeartbeatRequest request = new TezHeartbeatRequest(cData.numUpdates, events,
// cData.cIdStr, cData.taId, cData.nextFromEventId, 50000);
TaskHeartbeatRequest request =
@@ -431,7 +431,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
List<TezEvent> events = Collections.singletonList(new TezEvent(
new TaskAttemptCompletedEvent(), new EventMetaData(
EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId),
- getContext().getClock().getTime()));
+ MockDAGAppMaster.this.getContext().getClock().getTime()));
TaskHeartbeatRequest request =
new TaskHeartbeatRequest(cData.cIdStr, cData.taId, events, cData.nextFromEventId, cData.nextPreRoutedFromEventId,
10000);
http://git-wip-us.apache.org/repos/asf/tez/blob/5fdbe04f/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 639c487..4d404b9 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -89,7 +89,7 @@ import org.junit.Test;
import org.mockito.ArgumentCaptor;
@SuppressWarnings("unchecked")
-// TODO TEZ-2003 Rename to TestTezTaskCommunicator
+// TODO TEZ-2003 (post) TEZ-2696 Rename to TestTezTaskCommunicator
public class TestTaskAttemptListenerImplTezDag {
private ApplicationId appId;
private ApplicationAttemptId appAttemptId;
@@ -153,8 +153,6 @@ public class TestTaskAttemptListenerImplTezDag {
Lists.newArrayList(
new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
.setUserPayload(defaultPayload)));
- TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
- TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
taskSpec = mock(TaskSpec.class);
doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
http://git-wip-us.apache.org/repos/asf/tez/blob/5fdbe04f/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
index 08efb3e..42b957b 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestFetcher.java
@@ -273,7 +273,7 @@ public class TestFetcher {
int partition = 42;
FetcherCallback callback = mock(FetcherCallback.class);
Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(callback, null, null,
- ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, true, HOST, PORT, false);
+ ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, false);
builder.assignWork(HOST, PORT, partition, Arrays.asList(srcAttempts));
Fetcher fetcher = spy(builder.build());
fetcher.populateRemainingMap(new LinkedList<InputAttemptIdentifier>(Arrays.asList(srcAttempts)));
[42/51] [abbrv] tez git commit: TEZ-2651. Pluggable services should
not extend AbstractService. (sseth)
Posted by ss...@apache.org.
TEZ-2651. Pluggable services should not extend AbstractService. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/f5da29df
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/f5da29df
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/f5da29df
Branch: refs/heads/TEZ-2003
Commit: f5da29dfffe709d0b78ad69cf09a4c1b93b13411
Parents: 55a9eed
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jul 28 14:55:40 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:11 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../serviceplugins/api/ContainerLauncher.java | 18 ++++++++++--
.../apache/tez/dag/api/TaskCommunicator.java | 30 +++++++++++++++++---
.../tez/dag/api/TaskCommunicatorContext.java | 5 ++++
.../org/apache/tez/dag/app/DAGAppMaster.java | 4 +--
.../dag/app/TaskAttemptListenerImpTezDag.java | 16 ++++++-----
.../dag/app/TaskCommunicatorContextImpl.java | 9 ++++++
.../tez/dag/app/TezTaskCommunicatorImpl.java | 24 ++++++----------
.../dag/app/launcher/ContainerLauncherImpl.java | 6 ++--
.../app/launcher/ContainerLauncherRouter.java | 12 ++++++--
.../app/launcher/LocalContainerLauncher.java | 6 ++--
.../apache/tez/dag/app/MockDAGAppMaster.java | 6 ++--
.../app/TestTaskAttemptListenerImplTezDag.java | 8 +++---
.../TezTestServiceContainerLauncher.java | 6 ++--
.../TezTestServiceNoOpContainerLauncher.java | 2 +-
.../TezTestServiceTaskCommunicatorImpl.java | 29 +++++++++----------
16 files changed, 116 insertions(+), 66 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index a51669d..e57f76f 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -36,5 +36,6 @@ ALL CHANGES:
TEZ-2124. Change Node tracking to work per external container source.
TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
TEZ-2005. Define basic interface for pluggable TaskScheduler.
+ TEZ-2651. Pluggable services should not extend AbstractService.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
index 218edb6..8337dcb 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncher.java
@@ -17,6 +17,7 @@ package org.apache.tez.serviceplugins.api;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.common.ServicePluginLifecycle;
/**
* Plugin to allow custom container launchers to be written to launch containers on different types
@@ -25,18 +26,29 @@ import org.apache.hadoop.service.AbstractService;
@InterfaceAudience.Public
@InterfaceStability.Unstable
-public abstract class ContainerLauncher extends AbstractService {
+public abstract class ContainerLauncher implements ServicePluginLifecycle {
private final ContainerLauncherContext containerLauncherContext;
// TODO TEZ-2003 Simplify this by moving away from AbstractService. Potentially Guava AbstractService.
// A serviceInit(Configuration) is not likely to be very useful, and will expose unnecessary internal
// configuration to the services if populated with the AM Configuration
- public ContainerLauncher(String name, ContainerLauncherContext containerLauncherContext) {
- super(name);
+ public ContainerLauncher(ContainerLauncherContext containerLauncherContext) {
this.containerLauncherContext = containerLauncherContext;
}
+ @Override
+ public void initialize() throws Exception {
+ }
+
+ @Override
+ public void start() throws Exception {
+ }
+
+ @Override
+ public void shutdown() throws Exception {
+ }
+
public final ContainerLauncherContext getContext() {
return this.containerLauncherContext;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index 05e437c..f221414 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -18,9 +18,9 @@ import java.net.InetSocketAddress;
import java.util.Map;
import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.common.ServicePluginLifecycle;
import org.apache.tez.dag.api.event.VertexStateUpdate;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
@@ -28,11 +28,33 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.api.impl.TaskSpec;
// TODO TEZ-2003 Move this into the tez-api module
-public abstract class TaskCommunicator extends AbstractService {
- public TaskCommunicator(String name) {
- super(name);
+public abstract class TaskCommunicator implements ServicePluginLifecycle {
+
+ private final TaskCommunicatorContext taskCommunicatorContext;
+
+ public TaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+ this.taskCommunicatorContext = taskCommunicatorContext;
+ }
+
+ public TaskCommunicatorContext getContext() {
+ return taskCommunicatorContext;
+ }
+
+ @Override
+ public void initialize() throws Exception {
}
+ @Override
+ public void start() throws Exception {
+ }
+
+ @Override
+ public void shutdown() throws Exception {
+ }
+
+ // TODO Post TEZ-2003 Move this into the API module. Moving this requires abstractions for
+ // TaskSpec and related classes. (assuming that's efficient for execution)
+
// TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
// TODO When talking to an external service, this plugin implementer may need access to a host:port
public abstract void registerRunningContainer(ContainerId containerId, String hostname, int port);
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index b6e63f7..ab32ec1 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -18,6 +18,7 @@ import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -35,6 +36,9 @@ public interface TaskCommunicatorContext {
// TODO TEZ-2003 Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
+ // TODO TEZ-2003 To be replaced by getInitialPayload
+ Configuration getInitialConfiguration();
+
ApplicationAttemptId getApplicationAttemptId();
Credentials getCredentials();
@@ -42,6 +46,7 @@ public interface TaskCommunicatorContext {
boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
// TODO TEZ-2003 Split the heartbeat API to a liveness check and a status update
+ // KKK Rename this API
TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
boolean isKnownContainer(ContainerId containerId);
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index ef27ddf..f3914d8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -1047,8 +1047,8 @@ public class DAGAppMaster extends AbstractService {
String[] taskCommunicatorClasses,
boolean isLocal) {
TaskAttemptListener lis =
- new TaskAttemptListenerImpTezDag(context, thh, chh, jobTokenSecretManager,
- taskCommunicatorClasses, isLocal);
+ new TaskAttemptListenerImpTezDag(context, thh, chh,
+ taskCommunicatorClasses, amConf, isLocal);
return lis;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 47b63dd..599c208 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentMap;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.collections4.ListUtils;
+import org.apache.hadoop.conf.Configuration;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
@@ -61,7 +62,6 @@ import org.apache.tez.dag.app.rm.container.AMContainerTask;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.runtime.api.impl.TezEvent;
-import org.apache.tez.common.security.JobTokenSecretManager;
@SuppressWarnings("unchecked")
@@ -75,6 +75,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
private final AppContext context;
private final TaskCommunicator[] taskCommunicators;
private final TaskCommunicatorContext[] taskCommunicatorContexts;
+ protected final ServicePluginLifecycleAbstractService []taskCommunicatorServiceWrappers;
protected final TaskHeartbeatHandler taskHeartbeatHandler;
protected final ContainerHeartbeatHandler containerHeartbeatHandler;
@@ -99,9 +100,8 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
public TaskAttemptListenerImpTezDag(AppContext context,
TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
- // TODO TEZ-2003 pre-merge. Remove reference to JobTokenSecretManager.
- JobTokenSecretManager jobTokenSecretManager,
String [] taskCommunicatorClassIdentifiers,
+ Configuration conf,
boolean isPureLocalMode) {
super(TaskAttemptListenerImpTezDag.class.getName());
this.context = context;
@@ -118,9 +118,11 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorClassIdentifiers.length];
+ this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorClassIdentifiers.length];
for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
- taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, i);
+ taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, conf, i);
taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
+ taskCommunicatorServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskCommunicators[i]);
}
// TODO TEZ-2118 Start using taskCommunicator indices properly
}
@@ -129,15 +131,15 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
public void serviceStart() {
// TODO Why is init tied to serviceStart
for (int i = 0 ; i < taskCommunicators.length ; i++) {
- taskCommunicators[i].init(getConfig());
- taskCommunicators[i].start();
+ taskCommunicatorServiceWrappers[i].init(getConfig());
+ taskCommunicatorServiceWrappers[i].start();
}
}
@Override
public void serviceStop() {
for (int i = 0 ; i < taskCommunicators.length ; i++) {
- taskCommunicators[i].stop();
+ taskCommunicatorServiceWrappers[i].stop();
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index 50e006d..035db93 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -23,6 +23,7 @@ import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -48,14 +49,17 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
private final int taskCommunicatorIndex;
private final ReentrantReadWriteLock.ReadLock dagChangedReadLock;
private final ReentrantReadWriteLock.WriteLock dagChangedWriteLock;
+ private final Configuration conf;
private DAG dag;
public TaskCommunicatorContextImpl(AppContext appContext,
TaskAttemptListenerImpTezDag taskAttemptListener,
+ Configuration conf,
int taskCommunicatorIndex) {
this.context = appContext;
this.taskAttemptListener = taskAttemptListener;
+ this.conf = conf;
this.taskCommunicatorIndex = taskCommunicatorIndex;
ReentrantReadWriteLock dagChangedLock = new ReentrantReadWriteLock();
@@ -64,6 +68,11 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
}
@Override
+ public Configuration getInitialConfiguration() {
+ return conf;
+ }
+
+ @Override
public ApplicationAttemptId getApplicationAttemptId() {
return context.getApplicationAttemptId();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 0374022..93b5b43 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -67,7 +67,6 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
private static final ContainerTask TASK_FOR_INVALID_JVM = new ContainerTask(
null, true, null, null, false);
- private final TaskCommunicatorContext taskCommunicatorContext;
private final TezTaskUmbilicalProtocol taskUmbilical;
protected final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
@@ -116,25 +115,24 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
* Construct the service.
*/
public TezTaskCommunicatorImpl(TaskCommunicatorContext taskCommunicatorContext) {
- super(TezTaskCommunicatorImpl.class.getName());
- this.taskCommunicatorContext = taskCommunicatorContext;
+ super(taskCommunicatorContext);
this.taskUmbilical = new TezTaskUmbilicalProtocolImpl();
- this.tokenIdentifier = this.taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString();
+ this.tokenIdentifier = taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString();
this.sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
}
@Override
- public void serviceStart() {
+ public void start() {
startRpcServer();
}
@Override
- public void serviceStop() {
+ public void shutdown() {
stopRpcServer();
}
protected void startRpcServer() {
- Configuration conf = getConfig();
+ Configuration conf = getContext().getInitialConfiguration();
try {
JobTokenSecretManager jobTokenSecretManager =
new JobTokenSecretManager();
@@ -281,10 +279,6 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
return sessionToken;
}
- protected TaskCommunicatorContext getTaskCommunicatorContext() {
- return taskCommunicatorContext;
- }
-
public TezTaskUmbilicalProtocol getUmbilical() {
return this.taskUmbilical;
}
@@ -305,7 +299,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
}
task = getContainerTask(containerId);
if (task != null && !task.shouldDie()) {
- taskCommunicatorContext
+ getContext()
.taskStartedRemotely(task.getTaskSpec().getTaskAttemptID(), containerId);
}
}
@@ -317,7 +311,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
@Override
public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException {
- return taskCommunicatorContext.canCommit(taskAttemptId);
+ return getContext().canCommit(taskAttemptId);
}
@Override
@@ -370,7 +364,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
TaskHeartbeatRequest tRequest = new TaskHeartbeatRequest(request.getContainerIdentifier(),
request.getCurrentTaskAttemptID(), request.getEvents(), request.getStartIndex(),
request.getPreRoutedStartIndex(), request.getMaxEvents());
- tResponse = taskCommunicatorContext.heartbeat(tRequest);
+ tResponse = getContext().heartbeat(tRequest);
}
TezHeartbeatResponse response = new TezHeartbeatResponse();
response.setLastRequestId(requestId);
@@ -402,7 +396,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
ContainerInfo containerInfo = registeredContainers.get(containerId);
ContainerTask task = null;
if (containerInfo == null) {
- if (taskCommunicatorContext.isKnownContainer(containerId)) {
+ if (getContext().isKnownContainer(containerId)) {
LOG.info("Container with id: " + containerId
+ " is valid, but no longer registered, and will be killed");
} else {
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
index fe0178c..34c7bc0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
@@ -223,7 +223,7 @@ public class ContainerLauncherImpl extends ContainerLauncher {
}
public ContainerLauncherImpl(ContainerLauncherContext containerLauncherContext) {
- super(ContainerLauncherImpl.class.getName(), containerLauncherContext);
+ super(containerLauncherContext);
this.conf = new Configuration(containerLauncherContext.getInitialConfiguration());
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
@@ -235,7 +235,7 @@ public class ContainerLauncherImpl extends ContainerLauncher {
}
@Override
- public void serviceStart() {
+ public void start() {
// pass a copy of config to ContainerManagementProtocolProxy until YARN-3497 is fixed
cmProxy =
new ContainerManagementProtocolProxy(conf);
@@ -307,7 +307,7 @@ public class ContainerLauncherImpl extends ContainerLauncher {
}
@Override
- public void serviceStop() {
+ public void shutdown() {
if(!serviceStopped.compareAndSet(false, true)) {
LOG.info("Ignoring multiple stops");
return;
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 9f741cf..7c6a6a4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -45,6 +46,7 @@ public class ContainerLauncherRouter extends AbstractService
private final ContainerLauncher containerLaunchers[];
private final ContainerLauncherContext containerLauncherContexts[];
+ protected final ServicePluginLifecycleAbstractService[] containerLauncherServiceWrappers;
private final AppContext appContext;
@VisibleForTesting
@@ -53,6 +55,8 @@ public class ContainerLauncherRouter extends AbstractService
this.appContext = context;
containerLaunchers = new ContainerLauncher[] {containerLauncher};
containerLauncherContexts = new ContainerLauncherContext[] {containerLauncher.getContext()};
+ containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[]{
+ new ServicePluginLifecycleAbstractService(containerLauncher)};
}
// Accepting conf to setup final parameters, if required.
@@ -75,6 +79,7 @@ public class ContainerLauncherRouter extends AbstractService
}
containerLauncherContexts = new ContainerLauncherContext[containerLauncherClassIdentifiers.length];
containerLaunchers = new ContainerLauncher[containerLauncherClassIdentifiers.length];
+ containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[containerLauncherClassIdentifiers.length];
for (int i = 0; i < containerLauncherClassIdentifiers.length; i++) {
@@ -82,6 +87,7 @@ public class ContainerLauncherRouter extends AbstractService
containerLauncherContexts[i] = containerLauncherContext;
containerLaunchers[i] = createContainerLauncher(containerLauncherClassIdentifiers[i], context,
containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode, conf);
+ containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
}
}
@@ -130,21 +136,21 @@ public class ContainerLauncherRouter extends AbstractService
@Override
public void serviceInit(Configuration conf) {
for (int i = 0 ; i < containerLaunchers.length ; i++) {
- ((AbstractService) containerLaunchers[i]).init(conf);
+ containerLauncherServiceWrappers[i].init(conf);
}
}
@Override
public void serviceStart() {
for (int i = 0 ; i < containerLaunchers.length ; i++) {
- ((AbstractService) containerLaunchers[i]).start();
+ containerLauncherServiceWrappers[i].start();
}
}
@Override
public void serviceStop() {
for (int i = 0 ; i < containerLaunchers.length ; i++) {
- ((AbstractService) containerLaunchers[i]).stop();
+ containerLauncherServiceWrappers[i].stop();
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index a1b8e29..3975111 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -111,7 +111,7 @@ public class LocalContainerLauncher extends ContainerLauncher {
// starts up. It's not possible to set these up via a static payload.
// Will need some kind of mechanism to dynamically crate payloads / bind to parameters
// after the AM starts up.
- super(LocalContainerLauncher.class.getName(), containerLauncherContext);
+ super(containerLauncherContext);
this.context = context;
this.tal = taskAttemptListener;
this.workingDirectory = workingDirectory;
@@ -139,14 +139,14 @@ public class LocalContainerLauncher extends ContainerLauncher {
}
@Override
- public void serviceStart() throws Exception {
+ public void start() throws Exception {
eventHandlingThread =
new Thread(new TezSubTaskRunner(), "LocalContainerLauncher-SubTaskRunner");
eventHandlingThread.start();
}
@Override
- public void serviceStop() throws Exception {
+ public void shutdown() throws Exception {
if (!serviceStopped.compareAndSet(false, true)) {
LOG.info("Service Already stopped. Ignoring additional stop");
return;
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 3c3c6a7..21ae5f7 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -139,7 +139,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
public MockContainerLauncher(AtomicBoolean goFlag,
ContainerLauncherContext containerLauncherContext) {
- super("MockContainerLauncher", containerLauncherContext);
+ super(containerLauncherContext);
this.goFlag = goFlag;
}
@@ -182,7 +182,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
}
@Override
- public void serviceStart() throws Exception {
+ public void start() throws Exception {
taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
taskCommunicator = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator(0);
eventHandlingThread = new Thread(this);
@@ -199,7 +199,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
}
@Override
- public void serviceStop() throws Exception {
+ public void shutdown() throws Exception {
if (eventHandlingThread != null) {
eventHandlingThread.interrupt();
eventHandlingThread.join(2000l);
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index df643e4..41a7373 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -299,7 +299,7 @@ public class TestTaskAttemptListenerImplTezDag {
sessionToken.setService(identifier.getJobId());
TokenCache.setSessionToken(sessionToken, credentials);
taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, conf, false);
// no exception happen, should started properly
taskAttemptListener.init(conf);
taskAttemptListener.start();
@@ -319,7 +319,7 @@ public class TestTaskAttemptListenerImplTezDag {
conf.set(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE, port + "-" + port);
taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, conf, false);
taskAttemptListener.init(conf);
taskAttemptListener.start();
int resultedPort = taskAttemptListener.getTaskCommunicator(0).getAddress().getPort();
@@ -375,10 +375,10 @@ public class TestTaskAttemptListenerImplTezDag {
public TaskAttemptListenerImplForTest(AppContext context,
TaskHeartbeatHandler thh,
ContainerHeartbeatHandler chh,
- JobTokenSecretManager jobTokenSecretManager,
String[] taskCommunicatorClassIdentifiers,
+ Configuration conf,
boolean isPureLocalMode) {
- super(context, thh, chh, jobTokenSecretManager, taskCommunicatorClassIdentifiers,
+ super(context, thh, chh, taskCommunicatorClassIdentifiers, conf,
isPureLocalMode);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index dbf5054..85f9415 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -54,7 +54,7 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
// Configuration passed in here to set up final parameters
public TezTestServiceContainerLauncher(ContainerLauncherContext containerLauncherContext) {
- super(TezTestServiceContainerLauncher.class.getName(), containerLauncherContext);
+ super(containerLauncherContext);
int numThreads = getContext().getInitialConfiguration().getInt(
TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS,
TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT);
@@ -69,13 +69,13 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
}
@Override
- public void serviceStart() {
+ public void start() {
communicator.init(getContext().getInitialConfiguration());
communicator.start();
}
@Override
- public void serviceStop() {
+ public void shutdown() {
communicator.stop();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
index d3743e1..7b42296 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
@@ -27,7 +27,7 @@ public class TezTestServiceNoOpContainerLauncher extends ContainerLauncher {
public TezTestServiceNoOpContainerLauncher(ContainerLauncherContext containerLauncherContext) {
- super(TezTestServiceNoOpContainerLauncher.class.getName(), containerLauncherContext);
+ super(containerLauncherContext);
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/f5da29df/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index 444498e..078ea79 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -23,7 +23,6 @@ import java.util.concurrent.RejectedExecutionException;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.Credentials;
@@ -75,20 +74,20 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
}
@Override
- public void serviceInit(Configuration conf) throws Exception {
- super.serviceInit(conf);
- this.communicator.init(conf);
+ public void initialize() throws Exception {
+ super.initialize();
+ this.communicator.init(getContext().getInitialConfiguration());
}
@Override
- public void serviceStart() {
- super.serviceStart();
+ public void start() {
+ super.start();
this.communicator.start();
}
@Override
- public void serviceStop() {
- super.serviceStop();
+ public void shutdown() {
+ super.shutdown();
this.communicator.stop();
}
@@ -132,7 +131,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
}
// Have to register this up front right now. Otherwise, it's possible for the task to start
// sending out status/DONE/KILLED/FAILED messages before TAImpl knows how to handle them.
- getTaskCommunicatorContext()
+ getContext()
.taskStartedRemotely(taskSpec.getTaskAttemptID(), containerId);
communicator.submitWork(requestProto, host, port,
new TezTestServiceCommunicator.ExecuteRequestCallback<SubmitWorkResponseProto>() {
@@ -154,19 +153,19 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
RemoteException re = (RemoteException) t;
String message = re.toString();
if (message.contains(RejectedExecutionException.class.getName())) {
- getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
+ getContext().taskKilled(taskSpec.getTaskAttemptID(),
TaskAttemptEndReason.SERVICE_BUSY, "Service Busy");
} else {
- getTaskCommunicatorContext()
+ getContext()
.taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
t.toString());
}
} else {
if (t instanceof IOException) {
- getTaskCommunicatorContext().taskKilled(taskSpec.getTaskAttemptID(),
+ getContext().taskKilled(taskSpec.getTaskAttemptID(),
TaskAttemptEndReason.COMMUNICATION_ERROR, "Communication Error");
} else {
- getTaskCommunicatorContext()
+ getContext()
.taskFailed(taskSpec.getTaskAttemptID(), TaskAttemptEndReason.OTHER,
t.getMessage());
}
@@ -191,11 +190,11 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
builder.setAmPort(getAddress().getPort());
Credentials taskCredentials = new Credentials();
// Credentials can change across DAGs. Ideally construct only once per DAG.
- taskCredentials.addAll(getTaskCommunicatorContext().getCredentials());
+ taskCredentials.addAll(getContext().getCredentials());
ByteBuffer credentialsBinary = credentialMap.get(taskSpec.getDAGName());
if (credentialsBinary == null) {
- credentialsBinary = serializeCredentials(getTaskCommunicatorContext().getCredentials());
+ credentialsBinary = serializeCredentials(getContext().getCredentials());
credentialMap.putIfAbsent(taskSpec.getDAGName(), credentialsBinary.duplicate());
} else {
credentialsBinary = credentialsBinary.duplicate();
[26/51] [abbrv] tez git commit: TEZ-2434. Allow tasks to be killed in
the runtime. (sseth)
Posted by ss...@apache.org.
TEZ-2434. Allow tasks to be killed in the runtime. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/b51e271e
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/b51e271e
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/b51e271e
Branch: refs/heads/TEZ-2003
Commit: b51e271ef2e039dd2911b1ba0fcf982a7c7954da
Parents: d8fb6ad
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon May 11 23:34:43 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/runtime/task/EndReason.java | 29 ++
.../tez/runtime/task/TaskRunner2Callable.java | 132 ++++++
.../tez/runtime/task/TaskRunner2Result.java | 48 ++
.../org/apache/tez/runtime/task/TezChild.java | 20 +-
.../apache/tez/runtime/task/TezTaskRunner.java | 1 +
.../apache/tez/runtime/task/TezTaskRunner2.java | 434 +++++++++++++++++++
7 files changed, 655 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/b51e271e/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 66c110f..5d2e40a 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -24,5 +24,6 @@ ALL CHANGES:
TEZ-2420. TaskRunner returning before executing the task.
TEZ-2433. Fixes after rebase 05/08
TEZ-2438. tez-tools version in the branch is incorrect.
+ TEZ-2434. Allow tasks to be killed in the Runtime.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/b51e271e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/EndReason.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/EndReason.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/EndReason.java
new file mode 100644
index 0000000..8dc7a87
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/EndReason.java
@@ -0,0 +1,29 @@
+/*
+ * 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.tez.runtime.task;
+
+public enum EndReason {
+ SUCCESS(false),
+ CONTAINER_STOP_REQUESTED(false),
+ KILL_REQUESTED(true),
+ COMMUNICATION_FAILURE(false),
+ TASK_ERROR(false);
+
+ private final boolean isActionable;
+
+ EndReason(boolean isActionable) {
+ this.isActionable = isActionable;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/b51e271e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
new file mode 100644
index 0000000..7315bbd
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Callable.java
@@ -0,0 +1,132 @@
+/*
+ * 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.tez.runtime.task;
+
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.tez.common.CallableWithNdc;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is responsible for running a {@link LogicalIOProcessorRuntimeTask}.
+ * It does not worry about reporting errors, heartbeats etc.
+ *
+ * Returns success / interrupt / failure status via it's return parameter.
+ *
+ * It's the responsibility of the invoker to handle whatever exceptions may be generated by this.
+ */
+public class TaskRunner2Callable extends CallableWithNdc<TaskRunner2Callable.TaskRunner2CallableResult> {
+
+
+
+ private static final Logger LOG = LoggerFactory.getLogger(TaskRunner2Callable.class);
+
+ private final LogicalIOProcessorRuntimeTask task;
+ private final UserGroupInformation ugi;
+ private final AtomicBoolean stopRequested = new AtomicBoolean(false);
+
+ private volatile Thread ownThread;
+
+ public TaskRunner2Callable(LogicalIOProcessorRuntimeTask task,
+ UserGroupInformation ugi) {
+ this.task = task;
+ this.ugi = ugi;
+ }
+
+ @Override
+ public TaskRunner2CallableResult callInternal() throws Exception {
+ ownThread = Thread.currentThread();
+ if (stopRequested.get()) {
+ return new TaskRunner2CallableResult(null);
+ }
+ try {
+ return ugi.doAs(new PrivilegedExceptionAction<TaskRunner2CallableResult>() {
+ @Override
+ public TaskRunner2CallableResult run() throws Exception {
+ if (stopRequested.get() || Thread.currentThread().isInterrupted()) {
+ return new TaskRunner2CallableResult(null);
+ }
+ LOG.info("Initializing task" + ", taskAttemptId=" + task.getTaskAttemptID());
+ task.initialize();
+
+ if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) {
+ LOG.info("Running task, taskAttemptId=" + task.getTaskAttemptID());
+ task.run();
+ } else {
+ LOG.info("Stopped before running the processor.");
+ return new TaskRunner2CallableResult(null);
+ }
+
+ if (!stopRequested.get() && !Thread.currentThread().isInterrupted()) {
+ LOG.info("Closing task, taskAttemptId=" + task.getTaskAttemptID());
+ task.close();
+ task.setFrameworkCounters();
+ } else {
+ LOG.info("Stopped before closing the processor");
+ return new TaskRunner2CallableResult(null);
+ }
+ LOG.info("Task completed, taskAttemptId=" + task.getTaskAttemptID() + ", askedToStop=" + stopRequested.get());
+
+
+ return new TaskRunner2CallableResult(null);
+ }
+ });
+ } catch (Throwable t) {
+ if (t instanceof UndeclaredThrowableException) {
+ t = t.getCause();
+ }
+ return new TaskRunner2CallableResult(t);
+ } finally {
+ // If a stop was requested. Make sure the interrupt status is set during the cleanup.
+
+ // One drawback of not communicating out from here is that task complete messages will only
+ // be sent out after cleanup is complete.
+ // For a successful task, however, this should be almost no delay since close has already happened.
+ maybeFixInterruptStatus();
+ LOG.info("Cleaning up task {}, stopRequested={}", task.getTaskAttemptID(), stopRequested.get());
+ task.cleanup();
+ }
+ }
+
+ private void maybeFixInterruptStatus() {
+ if (stopRequested.get() && !Thread.currentThread().isInterrupted()) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+
+ public void interruptTask() {
+ // Ensure the task is only interrupted once.
+ if (!stopRequested.getAndSet(true)) {
+ if (ownThread != null) {
+ ownThread.interrupt();
+ }
+ }
+ }
+
+ public static class TaskRunner2CallableResult {
+ final Throwable error;
+
+ public TaskRunner2CallableResult(Throwable error) {
+ this.error = error;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/b51e271e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Result.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Result.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Result.java
new file mode 100644
index 0000000..07b32ce
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskRunner2Result.java
@@ -0,0 +1,48 @@
+/*
+ * 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.tez.runtime.task;
+
+public class TaskRunner2Result {
+ final EndReason endReason;
+ final Throwable error;
+ final boolean containerShutdownRequested;
+
+ public TaskRunner2Result(EndReason endReason, Throwable error, boolean containerShutdownRequested) {
+ this.endReason = endReason;
+ this.error = error;
+ this.containerShutdownRequested = containerShutdownRequested;
+ }
+
+ public EndReason getEndReason() {
+ return endReason;
+ }
+
+ public Throwable getError() {
+ return error;
+ }
+
+ public boolean isContainerShutdownRequested() {
+ return containerShutdownRequested;
+ }
+
+ @Override
+ public String toString() {
+ return "TaskRunner2Result{" +
+ "endReason=" + endReason +
+ ", error=" + error +
+ ", containerShutdownRequested=" + containerShutdownRequested +
+ '}';
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/b51e271e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index 4c8bebc..fff39a0 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -248,27 +248,27 @@ public class TezChild {
cleanupOnTaskChanged(containerTask);
// Execute the Actual Task
- TezTaskRunner taskRunner = new TezTaskRunner(defaultConf, childUGI,
+ TezTaskRunner2 taskRunner = new TezTaskRunner2(defaultConf, childUGI,
localDirs, containerTask.getTaskSpec(), appAttemptNumber,
serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap, taskReporter,
executor, objectRegistry, pid, executionContext, memAvailable);
boolean shouldDie;
try {
- shouldDie = !taskRunner.run();
+ TaskRunner2Result result = taskRunner.run();
+ shouldDie = result.isContainerShutdownRequested();
if (shouldDie) {
LOG.info("Got a shouldDie notification via heartbeats for container {}. Shutting down", containerIdString);
shutdown();
return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.SUCCESS, null,
"Asked to die by the AM");
}
- } catch (IOException e) {
- handleError(e);
- return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
- e, "TaskExecutionFailure: " + e.getMessage());
- } catch (TezException e) {
- handleError(e);
- return new ContainerExecutionResult(ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
- e, "TaskExecutionFailure: " + e.getMessage());
+ if (result.getError() != null) {
+ Throwable e = result.getError();
+ handleError(result.getError());
+ return new ContainerExecutionResult(
+ ContainerExecutionResult.ExitStatus.EXECUTION_FAILURE,
+ e, "TaskExecutionFailure: " + e.getMessage());
+ }
} finally {
FileSystem.closeAllForUGI(childUGI);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/b51e271e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index dd4620a..a82d87b 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -106,6 +106,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
if (!Thread.currentThread().isInterrupted()) {
taskFuture = executor.submit(callable);
} else {
+ taskReporter.unregisterTask(task.getTaskAttemptID());
return isShutdownRequested();
}
try {
http://git-wip-us.apache.org/repos/asf/tez/blob/b51e271e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
new file mode 100644
index 0000000..73e5c76
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -0,0 +1,434 @@
+/*
+ * 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.tez.runtime.task;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSError;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.ExecutionContext;
+import org.apache.tez.runtime.api.ObjectRegistry;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
+import org.apache.tez.runtime.task.TaskRunner2Callable.TaskRunner2CallableResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TezTaskRunner2 {
+
+
+ private static final Logger LOG = LoggerFactory.getLogger(TezTaskRunner2.class);
+
+ private final LogicalIOProcessorRuntimeTask task;
+ private final UserGroupInformation ugi;
+
+ private final TaskReporterInterface taskReporter;
+ private final ListeningExecutorService executor;
+ private final UmbilicalAndErrorHandler umbilicalAndErrorHandler;
+
+ // TODO It may be easier to model this as a state machine.
+
+ // Indicates whether a kill has been requested.
+ private final AtomicBoolean killTaskRequested = new AtomicBoolean(false);
+
+ // Indicates whether a stop container has been requested.
+ private final AtomicBoolean stopContainerRequested = new AtomicBoolean(false);
+
+ // Indicates whether the task is complete.
+ private final AtomicBoolean taskComplete = new AtomicBoolean(false);
+
+ // Separate flag from firstException, since an error can be reported without an exception.
+ private final AtomicBoolean errorSeen = new AtomicBoolean(false);
+
+ private volatile EndReason firstEndReason = null;
+
+ // The first exception which caused the task to fail. This could come in from the
+ // TaskRunnerCallable, a failure to heartbeat, or a signalFatalError on the context.
+ private volatile Throwable firstException;
+ private volatile EventMetaData exceptionSourceInfo;
+ private final AtomicBoolean errorReporterToAm = new AtomicBoolean(false);
+
+ private boolean oobSignalErrorInProgress = false;
+ private final Lock oobSignalLock = new ReentrantLock();
+ private final Condition oobSignalCondition = oobSignalLock.newCondition();
+
+ private volatile long taskKillStartTime = 0;
+
+ // The callable which is being used to execute the task.
+ private volatile TaskRunner2Callable taskRunnerCallable;
+
+ public TezTaskRunner2(Configuration tezConf, UserGroupInformation ugi, String[] localDirs,
+ TaskSpec taskSpec, int appAttemptNumber,
+ Map<String, ByteBuffer> serviceConsumerMetadata,
+ Map<String, String> serviceProviderEnvMap,
+ Multimap<String, String> startedInputsMap,
+ TaskReporterInterface taskReporter, ListeningExecutorService executor,
+ ObjectRegistry objectRegistry, String pid,
+ ExecutionContext executionContext, long memAvailable) throws
+ IOException {
+ this.ugi = ugi;
+ this.taskReporter = taskReporter;
+ this.executor = executor;
+ this.umbilicalAndErrorHandler = new UmbilicalAndErrorHandler();
+ this.task = new LogicalIOProcessorRuntimeTask(taskSpec, appAttemptNumber, tezConf, localDirs,
+ umbilicalAndErrorHandler, serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap,
+ objectRegistry, pid, executionContext, memAvailable);
+ }
+
+ /**
+ * Throws an exception only when there was a communication error reported by
+ * the TaskReporter.
+ *
+ * Otherwise, this takes care of all communication with the AM for a a running task - which
+ * includes informing the AM about Failures and Success.
+ *
+ * If a kill request is made to the task, it will not communicate this information to
+ * the AM - since a task KILL is an external event, and whoever invoked it should
+ * be able to track it.
+ *
+ * @return
+ */
+ public TaskRunner2Result run() {
+ try {
+ ListenableFuture<TaskRunner2CallableResult> future = null;
+ synchronized (this) {
+ if (isRunningState()) {
+ // Safe to do this within a synchronized block because we're providing
+ // the handler on which the Reporter will communicate back. Assuming
+ // the register call doesn't end up hanging.
+ taskRunnerCallable = new TaskRunner2Callable(task, ugi);
+ taskReporter.registerTask(task, umbilicalAndErrorHandler);
+ future = executor.submit(taskRunnerCallable);
+ }
+ }
+
+ if (future == null) {
+ return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+ }
+
+ TaskRunner2CallableResult executionResult = null;
+ // The task started. Wait for it to complete.
+ try {
+ executionResult = future.get();
+ } catch (Throwable e) {
+ if (e instanceof ExecutionException) {
+ e = e.getCause();
+ }
+ synchronized (this) {
+ if (isRunningState()) {
+ trySettingEndReason(EndReason.TASK_ERROR);
+ registerFirstException(e, null);
+ LOG.warn("Exception from RunnerCallable", e);
+ }
+ }
+ }
+ if (executionResult != null) {
+ synchronized (this) {
+ if (isRunningState()) {
+ if (executionResult.error != null) {
+ trySettingEndReason(EndReason.TASK_ERROR);
+ registerFirstException(executionResult.error, null);
+ } else {
+ trySettingEndReason(EndReason.SUCCESS);
+ taskComplete.set(true);
+ }
+ }
+ }
+ }
+
+ switch (firstEndReason) {
+ case SUCCESS:
+ try {
+ taskReporter.taskSucceeded(task.getTaskAttemptID());
+ return logAndReturnEndResult(EndReason.SUCCESS, null, stopContainerRequested.get());
+ } catch (IOException e) {
+ // Comm failure. Task can't do much.
+ handleFinalStatusUpdateFailure(e, true);
+ return logAndReturnEndResult(EndReason.COMMUNICATION_FAILURE, e, stopContainerRequested.get());
+ } catch (TezException e) {
+ // Failure from AM. Task can't do much.
+ handleFinalStatusUpdateFailure(e, true);
+ return logAndReturnEndResult(EndReason.COMMUNICATION_FAILURE, e, stopContainerRequested.get());
+ }
+ case CONTAINER_STOP_REQUESTED:
+ // Don't need to send any more communication updates to the AM.
+ return logAndReturnEndResult(firstEndReason, null, stopContainerRequested.get());
+ case KILL_REQUESTED:
+ // Kill is currently not reported to the AM via the TaskRunner. Fix this when the umbilical
+ // supports an indication of kill, if required.
+ return logAndReturnEndResult(firstEndReason, null, stopContainerRequested.get());
+ case COMMUNICATION_FAILURE:
+ // Already seen a communication failure. There's no point trying to report another one.
+ return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+ case TASK_ERROR:
+ // Don't report an error again if it was reported via signalFatalError
+ if (errorReporterToAm.get()) {
+ return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+ } else {
+ String message;
+ if (firstException instanceof FSError) {
+ message = "Encountered an FSError while executing task: " + task.getTaskAttemptID();
+ } else if (firstException instanceof Error) {
+ message = "Encountered an Error while executing task: " + task.getTaskAttemptID();
+ } else {
+ message = "Failure while running task: " + task.getTaskAttemptID();
+ }
+ try {
+ taskReporter.taskFailed(task.getTaskAttemptID(), firstException, message, exceptionSourceInfo);
+ return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+ } catch (IOException e) {
+ // Comm failure. Task can't do much.
+ handleFinalStatusUpdateFailure(e, true);
+ return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+ } catch (TezException e) {
+ // Failure from AM. Task can't do much.
+ handleFinalStatusUpdateFailure(e, true);
+ return logAndReturnEndResult(firstEndReason, firstException, stopContainerRequested.get());
+ }
+ }
+ default:
+ LOG.error("Unexpected EndReason. File a bug");
+ return logAndReturnEndResult(EndReason.TASK_ERROR, new RuntimeException("Unexpected EndReason"), stopContainerRequested.get());
+
+ }
+ } finally {
+ // Clear the interrupted status of the blocking thread, in case it is set after the
+ // InterruptedException was invoked.
+ oobSignalLock.lock();
+ try {
+ while (oobSignalErrorInProgress) {
+ try {
+ oobSignalCondition.await();
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while waiting for OOB fatal error to complete");
+ Thread.currentThread().interrupt();
+ }
+ }
+ } finally {
+ oobSignalLock.unlock();
+ }
+ taskReporter.unregisterTask(task.getTaskAttemptID());
+ if (taskKillStartTime != 0) {
+ LOG.info("Time taken to interrupt task={}", (System.currentTimeMillis() - taskKillStartTime));
+ }
+ Thread.interrupted();
+ }
+ }
+
+ public void killTask() {
+ synchronized (this) {
+ if (isRunningState()) {
+ trySettingEndReason(EndReason.KILL_REQUESTED);
+ if (taskRunnerCallable != null) {
+ taskKillStartTime = System.currentTimeMillis();
+ taskRunnerCallable.interruptTask();
+ }
+ }
+ }
+ }
+
+
+ // Checks and changes on these states should happen within a synchronized block,
+ // to ensure the first event is the one that is captured and causes specific behaviour.
+ private boolean isRunningState() {
+ return !taskComplete.get() && !killTaskRequested.get() && !stopContainerRequested.get() &&
+ !errorSeen.get();
+ }
+
+ class UmbilicalAndErrorHandler implements TezUmbilical, ErrorReporter {
+
+ @Override
+ public void addEvents(Collection<TezEvent> events) {
+ // Incoming events from the running task.
+ // Only add these if the task is running.
+ if (isRunningState()) {
+ taskReporter.addEvents(task.getTaskAttemptID(), events);
+ }
+ }
+
+ @Override
+ public void signalFatalError(TezTaskAttemptID taskAttemptID, Throwable t, String message,
+ EventMetaData sourceInfo) {
+ // Fatal error reported by the task.
+ boolean isFirstError = false;
+ synchronized (TezTaskRunner2.this) {
+ if (isRunningState()) {
+ if (trySettingEndReason(EndReason.TASK_ERROR)) {
+ if (t == null) {
+ t = new RuntimeException(
+ message == null ? "FatalError: No user message or exception specified" : message);
+ }
+ registerFirstException(t, sourceInfo);
+ LOG.info("Received notification of a fatal error which will cause the task to die", t);
+ isFirstError = true;
+ errorReporterToAm.set(true);
+ oobSignalErrorInProgress = true;
+ } else {
+ LOG.info(
+ "Ignoring fatal error since the task has ended for reason: {}. IgnoredError: {} ",
+ firstEndReason, (t == null ? message : t.getMessage()));
+ }
+ }
+ }
+
+ // Informing the TaskReporter here because the running task may not be interruptable.
+ // Has to be outside the lock.
+ if (isFirstError) {
+ killTask();
+ try {
+ taskReporter.taskFailed(taskAttemptID, t, getTaskDiagnosticsString(t, message), sourceInfo);
+ } catch (IOException e) {
+ // Comm failure. Task can't do much. The main exception is already registered.
+ handleFinalStatusUpdateFailure(e, true);
+ } catch (TezException e) {
+ // Failure from AM. Task can't do much. The main exception is already registered.
+ handleFinalStatusUpdateFailure(e, true);
+ } finally {
+ oobSignalLock.lock();
+ try {
+ // This message is being sent outside of the main thread, which may end up completing before
+ // this thread runs. Make sure the main run thread does not end till this completes.
+ oobSignalErrorInProgress = false;
+ oobSignalCondition.signal();
+ } finally {
+ oobSignalLock.unlock();
+ }
+ }
+ }
+ }
+
+ @Override
+ public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException {
+ // Task checking whether it can commit.
+
+ // Not getting a lock here. It should be alright for the to check with the reporter
+ // on whether a task can commit.
+ if (isRunningState()) {
+ return taskReporter.canCommit(taskAttemptID);
+ // If there's a communication failure here, let it propagate through to the task.
+ // which may throw it back or handle it appropriately.
+ } else {
+ // Don't throw an error since the task is already in the process of shutting down.
+ LOG.info("returning canCommit=false since task is not in a running state");
+ return false;
+ }
+ }
+
+
+ @Override
+ public void reportError(Throwable t) {
+ // Umbilical reporting an error during heartbeat
+ boolean isFirstError = false;
+ synchronized (TezTaskRunner2.this) {
+ if (isRunningState()) {
+ LOG.info("TaskReporter reporter error which will cause the task to fail", t);
+ if (trySettingEndReason(EndReason.COMMUNICATION_FAILURE)) {
+ registerFirstException(t, null);
+ isFirstError = true;
+ }
+ // A race is possible between a task succeeding, and a subsequent timed heartbeat failing.
+ // These errors can be ignored, since a task can only succeed if the synchronous taskSucceeded
+ // method does not throw an exception, in which case task success is registered with the AM.
+ // Leave subsequent heartbeat errors to the next entity to communicate using the TaskReporter
+ } else {
+ LOG.info("Ignoring Communication failure since task with id=" + task.getTaskAttemptID()
+ + " is already complete, is failing or has been asked to terminate");
+ }
+ }
+ // Since this error came from the taskReporter - there's no point attempting to report a failure back to it.
+ if (isFirstError) {
+ killTask();
+ }
+ }
+
+ @Override
+ public void shutdownRequested() {
+ // Umbilical informing about a shutdown request for the container.
+ boolean isFirstTerminate = false;
+ synchronized (TezTaskRunner2.this) {
+ isFirstTerminate = trySettingEndReason(EndReason.CONTAINER_STOP_REQUESTED);
+ // Respect stopContainerRequested since it can come in at any point, despite a previous failure.
+ stopContainerRequested.set(true);
+ }
+
+ if (isFirstTerminate) {
+ killTask();
+ }
+ }
+ }
+
+ private synchronized boolean trySettingEndReason(EndReason endReason) {
+ if (isRunningState()) {
+ firstEndReason = endReason;
+ return true;
+ }
+ return false;
+ }
+
+
+ private void registerFirstException(Throwable t, EventMetaData sourceInfo) {
+ Preconditions.checkState(isRunningState());
+ errorSeen.set(true);
+ firstException = t;
+ this.exceptionSourceInfo = sourceInfo;
+ }
+
+
+ private String getTaskDiagnosticsString(Throwable t, String message) {
+ String diagnostics;
+ if (t != null && message != null) {
+ diagnostics = "exceptionThrown=" + ExceptionUtils.getStackTrace(t) + ", errorMessage="
+ + message;
+ } else if (t == null && message == null) {
+ diagnostics = "Unknown error";
+ } else {
+ diagnostics = t != null ? "exceptionThrown=" + ExceptionUtils.getStackTrace(t)
+ : " errorMessage=" + message;
+ }
+ return diagnostics;
+ }
+
+ private TaskRunner2Result logAndReturnEndResult(EndReason endReason, Throwable firstError,
+ boolean stopContainerRequested) {
+ TaskRunner2Result result = new TaskRunner2Result(endReason, firstError, stopContainerRequested);
+ LOG.info("TaskRunnerResult for {} : {} ", task.getTaskAttemptID(), result);
+ return result;
+ }
+
+ private void handleFinalStatusUpdateFailure(Throwable t, boolean successReportAttempted) {
+ // TODO Ideally differentiate between FAILED/KILLED
+ LOG.warn("Failure while reporting state= {} to AM", (successReportAttempted ? "success" : "failure/killed"), t);
+ }
+}
\ No newline at end of file
[07/51] [abbrv] tez git commit: TEZ-2123. Fix component managers to
use pluggable components. Enable hybrid mode. (sseth)
Posted by ss...@apache.org.
TEZ-2123. Fix component managers to use pluggable components. Enable
hybrid mode. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/359eba29
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/359eba29
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/359eba29
Branch: refs/heads/TEZ-2003
Commit: 359eba295c0008ffdd2712ff609b066319ce92ef
Parents: b056821
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 11:59:03 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:24:43 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/dag/app/DAGAppMaster.java | 4 +-
.../apache/tez/dag/app/TaskAttemptListener.java | 12 +-
.../dag/app/TaskAttemptListenerImpTezDag.java | 27 ++--
.../tez/dag/app/dag/impl/TaskAttemptImpl.java | 4 +-
.../TezRootInputInitializerContextImpl.java | 2 +-
.../apache/tez/dag/app/dag/impl/VertexImpl.java | 2 +-
.../tez/dag/app/dag/impl/VertexManager.java | 2 +-
.../app/launcher/ContainerLauncherRouter.java | 2 +-
.../app/launcher/LocalContainerLauncher.java | 10 +-
.../rm/AMSchedulerEventDeallocateContainer.java | 7 +-
.../rm/AMSchedulerEventNodeBlacklistUpdate.java | 8 +-
.../tez/dag/app/rm/AMSchedulerEventTAEnded.java | 10 +-
.../dag/app/rm/LocalTaskSchedulerService.java | 19 ++-
.../tez/dag/app/rm/NMCommunicatorEvent.java | 12 +-
.../rm/NMCommunicatorLaunchRequestEvent.java | 11 +-
.../app/rm/NMCommunicatorStopRequestEvent.java | 4 +-
.../dag/app/rm/TaskSchedulerEventHandler.java | 151 ++++++++++++-----
.../tez/dag/app/rm/container/AMContainer.java | 3 +
.../AMContainerEventLaunchRequest.java | 15 +-
.../dag/app/rm/container/AMContainerImpl.java | 39 +++--
.../dag/app/rm/container/AMContainerMap.java | 4 +-
.../apache/tez/dag/app/rm/node/AMNodeImpl.java | 6 +-
.../apache/tez/dag/app/MockDAGAppMaster.java | 2 +-
.../app/TestTaskAttemptListenerImplTezDag.java | 25 ++-
.../tez/dag/app/dag/impl/TestTaskAttempt.java | 69 ++++----
.../tez/dag/app/dag/impl/TestVertexImpl.java | 8 +-
.../tez/dag/app/rm/TestContainerReuse.java | 34 ++--
.../tez/dag/app/rm/TestLocalTaskScheduler.java | 2 +-
.../app/rm/TestLocalTaskSchedulerService.java | 18 ++-
.../app/rm/TestTaskSchedulerEventHandler.java | 11 +-
.../dag/app/rm/TestTaskSchedulerHelpers.java | 2 +-
.../dag/app/rm/container/TestAMContainer.java | 108 +++++++------
.../app/rm/container/TestAMContainerMap.java | 6 +-
.../org/apache/tez/examples/JoinValidate.java | 30 +++-
.../TezTestServiceContainerLauncher.java | 5 +-
.../rm/TezTestServiceTaskSchedulerService.java | 100 ++----------
.../tez/examples/JoinValidateConfigured.java | 53 ++++++
.../tez/tests/TestExternalTezServices.java | 160 ++++++++++++++-----
39 files changed, 631 insertions(+), 357 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 4bfe08f..1a2264c 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -4,5 +4,6 @@ ALL CHANGES:
TEZ-2090. Add tests for jobs running in external services.
TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
TEZ-2122. Setup pluggable components at AM/Vertex level.
+ TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 431a8b2..a304b37 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -517,7 +517,7 @@ public class DAGAppMaster extends AbstractService {
this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
- taskSchedulerClassIdentifiers);
+ taskSchedulerClassIdentifiers, isLocal);
addIfService(taskSchedulerEventHandler, true);
if (enableWebUIService()) {
@@ -2341,6 +2341,7 @@ public class DAGAppMaster extends AbstractService {
// Tez default classnames are populated as TezConfiguration.TEZ_AM_SERVICE_PLUGINS_DEFAULT
private String[] parsePlugins(BiMap<String, Integer> pluginMap, String[] pluginStrings,
String context) {
+ // TODO TEZ-2003 Duplicate error checking - ideally in the client itself. Depends on the final API.
Preconditions.checkState(pluginStrings != null && pluginStrings.length > 0,
"Plugin strings should not be null or empty: " + context);
@@ -2378,6 +2379,7 @@ public class DAGAppMaster extends AbstractService {
}
pluginMap.put(identifierString, index);
classNames[index] = className;
+ index++;
}
return classNames;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
index 9caa7cf..e4dad27 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
@@ -22,6 +22,7 @@ import java.net.InetSocketAddress;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.app.rm.container.AMContainerTask;
import org.apache.tez.dag.records.TezTaskAttemptID;
/**
@@ -29,18 +30,17 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
*/
public interface TaskAttemptListener {
- InetSocketAddress getAddress();
+ void registerRunningContainer(ContainerId containerId, int taskCommId);
- void registerRunningContainer(ContainerId containerId);
-
- void registerTaskAttempt(AMContainerTask amContainerTask, ContainerId containerId);
+ void registerTaskAttempt(AMContainerTask amContainerTask, ContainerId containerId, int taskCommId);
- void unregisterRunningContainer(ContainerId containerId);
+ void unregisterRunningContainer(ContainerId containerId, int taskCommId);
- void unregisterTaskAttempt(TezTaskAttemptID attemptID);
+ void unregisterTaskAttempt(TezTaskAttemptID attemptID, int taskCommId);
void dagComplete(DAG dag);
void dagSubmitted();
+ TaskCommunicator getTaskCommunicator(int taskCommIndex);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 3d9abdf..05c4623 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -272,11 +272,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
return task.canCommit(taskAttemptId);
}
- @Override
- public InetSocketAddress getAddress() {
- return taskCommunicators[0].getAddress();
- }
-
// The TaskAttemptListener register / unregister methods in this class are not thread safe.
// The Tez framework should not invoke these methods from multiple threads.
@Override
@@ -296,7 +291,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
@Override
- public void registerRunningContainer(ContainerId containerId) {
+ public void registerRunningContainer(ContainerId containerId, int taskCommId) {
if (LOG.isDebugEnabled()) {
LOG.debug("ContainerId: " + containerId + " registered with TaskAttemptListener");
}
@@ -306,11 +301,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
"Multiple registrations for containerId: " + containerId);
}
NodeId nodeId = context.getAllContainers().get(containerId).getContainer().getNodeId();
- taskCommunicators[0].registerRunningContainer(containerId, nodeId.getHost(), nodeId.getPort());
+ taskCommunicators[taskCommId].registerRunningContainer(containerId, nodeId.getHost(),
+ nodeId.getPort());
}
@Override
- public void unregisterRunningContainer(ContainerId containerId) {
+ public void unregisterRunningContainer(ContainerId containerId, int taskCommId) {
if (LOG.isDebugEnabled()) {
LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId);
}
@@ -318,12 +314,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
if (containerInfo.taskAttemptId != null) {
registeredAttempts.remove(containerInfo.taskAttemptId);
}
- taskCommunicators[0].registerContainerEnd(containerId);
+ taskCommunicators[taskCommId].registerContainerEnd(containerId);
}
@Override
public void registerTaskAttempt(AMContainerTask amContainerTask,
- ContainerId containerId) {
+ ContainerId containerId, int taskCommId) {
ContainerInfo containerInfo = registeredContainers.get(containerId);
if (containerInfo == null) {
throw new TezUncheckedException("Registering task attempt: "
@@ -353,13 +349,13 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
+ amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+ " when already assigned to: " + containerIdFromMap);
}
- taskCommunicators[0].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
+ taskCommunicators[taskCommId].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
amContainerTask.haveCredentialsChanged());
}
@Override
- public void unregisterTaskAttempt(TezTaskAttemptID attemptId) {
+ public void unregisterTaskAttempt(TezTaskAttemptID attemptId, int taskCommId) {
ContainerId containerId = registeredAttempts.remove(attemptId);
if (containerId == null) {
LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
@@ -373,7 +369,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
// Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
registeredContainers.put(containerId, NULL_CONTAINER_INFO);
- taskCommunicators[0].unregisterRunningTaskAttempt(attemptId);
+ taskCommunicators[taskCommId].unregisterRunningTaskAttempt(attemptId);
+ }
+
+ @Override
+ public TaskCommunicator getTaskCommunicator(int taskCommIndex) {
+ return taskCommunicators[taskCommIndex];
}
private void pingContainerHeartbeatHandler(ContainerId containerId) {
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
index 806b977..1466817 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -1195,7 +1195,7 @@ public class TaskAttemptImpl implements TaskAttempt,
// Inform the scheduler
if (sendSchedulerEvent()) {
ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
- .getTaskAttemptState()));
+ .getTaskAttemptState(), ta.getVertex().getTaskSchedulerIdentifier()));
}
}
}
@@ -1277,7 +1277,7 @@ public class TaskAttemptImpl implements TaskAttempt,
// Inform the Scheduler.
ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
- TaskAttemptState.SUCCEEDED));
+ TaskAttemptState.SUCCEEDED, ta.getVertex().getTaskSchedulerIdentifier()));
// Inform the task.
ta.sendEvent(new TaskEventTAUpdate(ta.attemptId,
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
index d4ef4d5..4ca4024 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
@@ -96,7 +96,7 @@ public class TezRootInputInitializerContextImpl implements
@Override
public Resource getTotalAvailableResource() {
- return appContext.getTaskScheduler().getTotalResources();
+ return appContext.getTaskScheduler().getTotalResources(vertex.getTaskSchedulerIdentifier());
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index 0583a0b..7733ef9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -4536,7 +4536,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
eventHandler, getTotalTasks(),
appContext.getTaskScheduler().getNumClusterNodes(),
getTaskResource(),
- appContext.getTaskScheduler().getTotalResources());
+ appContext.getTaskScheduler().getTotalResources(taskSchedulerIdentifier));
List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
inputList = Lists.newArrayListWithCapacity(inputsWithInitializers.size());
for (String inputName : inputsWithInitializers) {
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
index 9476860..247b92f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
@@ -294,7 +294,7 @@ public class VertexManager {
@Override
public synchronized Resource getTotalAvailableResource() {
checkAndThrowIfDone();
- return appContext.getTaskScheduler().getTotalResources();
+ return appContext.getTaskScheduler().getTotalResources(managedVertex.getTaskSchedulerIdentifier());
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 621e4a8..4f9b5bf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -124,6 +124,6 @@ public class ContainerLauncherRouter extends AbstractService
@Override
public void handle(NMCommunicatorEvent event) {
- containerLaunchers[0].handle(event);
+ containerLaunchers[event.getLauncherId()].handle(event);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index d9d668f..3c27678 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -59,7 +59,6 @@ import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.TaskAttemptListener;
-import org.apache.tez.dag.app.TaskAttemptListenerImpTezDag;
import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
@@ -88,9 +87,9 @@ public class LocalContainerLauncher extends AbstractService implements
private static final Logger LOG = LoggerFactory.getLogger(LocalContainerLauncher.class);
private final AppContext context;
- private final TezTaskUmbilicalProtocol taskUmbilicalProtocol;
private final AtomicBoolean serviceStopped = new AtomicBoolean(false);
private final String workingDirectory;
+ private final TaskAttemptListener tal;
private final Map<String, String> localEnv = new HashMap<String, String>();
private final ExecutionContext executionContext;
private int numExecutors;
@@ -116,9 +115,8 @@ public class LocalContainerLauncher extends AbstractService implements
String workingDirectory) throws UnknownHostException {
super(LocalContainerLauncher.class.getName());
this.context = context;
- TaskAttemptListenerImpTezDag taListener = (TaskAttemptListenerImpTezDag)taskAttemptListener;
- TezTaskCommunicatorImpl taskComm = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator();
- this.taskUmbilicalProtocol = taskComm.getUmbilical();
+ this.tal = taskAttemptListener;
+
this.workingDirectory = workingDirectory;
AuxiliaryServiceHelper.setServiceDataIntoEnv(
ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ByteBuffer.allocate(4).putInt(0), localEnv);
@@ -223,7 +221,7 @@ public class LocalContainerLauncher extends AbstractService implements
tezChild =
createTezChild(context.getAMConf(), event.getContainerId(), tokenIdentifier,
context.getApplicationAttemptId().getAttemptId(), context.getLocalDirs(),
- taskUmbilicalProtocol,
+ ((TezTaskCommunicatorImpl)tal.getTaskCommunicator(event.getTaskCommId())).getUmbilical(),
TezCommonUtils.parseCredentialsBytes(event.getContainerLaunchContext().getTokens().array()));
} catch (InterruptedException e) {
handleLaunchFailed(e, event.getContainerId());
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java
index 1b51920..5270aa2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventDeallocateContainer.java
@@ -23,15 +23,20 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
public class AMSchedulerEventDeallocateContainer extends AMSchedulerEvent {
private final ContainerId containerId;
+ private final int schedulerId;
- public AMSchedulerEventDeallocateContainer(ContainerId containerId) {
+ public AMSchedulerEventDeallocateContainer(ContainerId containerId, int schedulerId) {
super(AMSchedulerEventType.S_CONTAINER_DEALLOCATE);
this.containerId = containerId;
+ this.schedulerId = schedulerId;
}
public ContainerId getContainerId() {
return this.containerId;
}
+ public int getSchedulerId() {
+ return schedulerId;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java
index ed7ebc3..679705a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventNodeBlacklistUpdate.java
@@ -23,14 +23,20 @@ import org.apache.hadoop.yarn.api.records.NodeId;
public class AMSchedulerEventNodeBlacklistUpdate extends AMSchedulerEvent {
private final NodeId nodeId;
+ private final int schedulerId;
- public AMSchedulerEventNodeBlacklistUpdate(NodeId nodeId, boolean add) {
+ public AMSchedulerEventNodeBlacklistUpdate(NodeId nodeId, boolean add, int schedulerId) {
super((add ? AMSchedulerEventType.S_NODE_BLACKLISTED
: AMSchedulerEventType.S_NODE_UNBLACKLISTED));
this.nodeId = nodeId;
+ this.schedulerId = schedulerId;
}
public NodeId getNodeId() {
return this.nodeId;
}
+
+ public int getSchedulerId() {
+ return schedulerId;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
index 90e76b7..2ace642 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTAEnded.java
@@ -26,14 +26,16 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
private final TaskAttempt attempt;
private final ContainerId containerId;
- private TaskAttemptState state;
+ private final TaskAttemptState state;
+ private final int schedulerId;
public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId,
- TaskAttemptState state) {
+ TaskAttemptState state, int schedulerId) {
super(AMSchedulerEventType.S_TA_ENDED);
this.attempt = attempt;
this.containerId = containerId;
this.state = state;
+ this.schedulerId = schedulerId;
}
public TezTaskAttemptID getAttemptID() {
@@ -51,4 +53,8 @@ public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
public ContainerId getUsedContainerId() {
return this.containerId;
}
+
+ public int getSchedulerId() {
+ return schedulerId;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
index 51d8b9d..72a074f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
@@ -34,6 +34,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
@@ -63,10 +64,11 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
final int appHostPort;
final String appTrackingUrl;
final AppContext appContext;
+ final long customContainerAppId;
public LocalTaskSchedulerService(TaskSchedulerAppCallback appClient,
ContainerSignatureMatcher containerSignatureMatcher, String appHostName,
- int appHostPort, String appTrackingUrl, AppContext appContext) {
+ int appHostPort, String appTrackingUrl, long customContainerAppId, AppContext appContext) {
super(LocalTaskSchedulerService.class.getName());
this.realAppClient = appClient;
this.appCallbackExecutor = createAppCallbackExecutorService();
@@ -78,6 +80,7 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
this.appContext = appContext;
taskRequestQueue = new PriorityBlockingQueue<TaskRequest>();
taskAllocations = new LinkedHashMap<Object, Container>();
+ this.customContainerAppId = customContainerAppId;
}
private ExecutorService createAppCallbackExecutorService() {
@@ -164,7 +167,7 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
protected AsyncDelegateRequestHandler createRequestHandler(Configuration conf) {
return new AsyncDelegateRequestHandler(taskRequestQueue,
- new LocalContainerFactory(appContext),
+ new LocalContainerFactory(appContext, customContainerAppId),
taskAllocations,
appClientDelegate,
conf);
@@ -195,17 +198,19 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
}
static class LocalContainerFactory {
- final AppContext appContext;
AtomicInteger nextId;
+ final ApplicationAttemptId customAppAttemptId;
- public LocalContainerFactory(AppContext appContext) {
- this.appContext = appContext;
+ public LocalContainerFactory(AppContext appContext, long appIdLong) {
this.nextId = new AtomicInteger(1);
+ ApplicationId appId = ApplicationId
+ .newInstance(appIdLong, appContext.getApplicationAttemptId().getApplicationId().getId());
+ this.customAppAttemptId = ApplicationAttemptId
+ .newInstance(appId, appContext.getApplicationAttemptId().getAttemptId());
}
public Container createContainer(Resource capability, Priority priority) {
- ApplicationAttemptId appAttemptId = appContext.getApplicationAttemptId();
- ContainerId containerId = ContainerId.newInstance(appAttemptId, nextId.getAndIncrement());
+ ContainerId containerId = ContainerId.newInstance(customAppAttemptId, nextId.getAndIncrement());
NodeId nodeId = NodeId.newInstance("127.0.0.1", 0);
String nodeHttpAddress = "127.0.0.1:0";
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
index 8bdeb28..f86894f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorEvent.java
@@ -28,13 +28,15 @@ public class NMCommunicatorEvent extends AbstractEvent<NMCommunicatorEventType>
private final ContainerId containerId;
private final NodeId nodeId;
private final Token containerToken;
+ private final int launcherId;
public NMCommunicatorEvent(ContainerId containerId, NodeId nodeId,
- Token containerToken, NMCommunicatorEventType type) {
+ Token containerToken, NMCommunicatorEventType type, int launcherId) {
super(type);
this.containerId = containerId;
this.nodeId = nodeId;
this.containerToken = containerToken;
+ this.launcherId = launcherId;
}
public ContainerId getContainerId() {
@@ -48,10 +50,14 @@ public class NMCommunicatorEvent extends AbstractEvent<NMCommunicatorEventType>
public Token getContainerToken() {
return this.containerToken;
}
-
+
+ public int getLauncherId() {
+ return launcherId;
+ }
+
public String toSrting() {
return super.toString() + " for container " + containerId + ", nodeId: "
- + nodeId;
+ + nodeId + ", launcherId: " + launcherId;
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
index c3b12c0..a38345c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorLaunchRequestEvent.java
@@ -25,13 +25,16 @@ public class NMCommunicatorLaunchRequestEvent extends NMCommunicatorEvent {
private final ContainerLaunchContext clc;
private final Container container;
+ // The task communicator index for the specific container being launched.
+ private final int taskCommId;
public NMCommunicatorLaunchRequestEvent(ContainerLaunchContext clc,
- Container container) {
+ Container container, int launcherId, int taskCommId) {
super(container.getId(), container.getNodeId(), container
- .getContainerToken(), NMCommunicatorEventType.CONTAINER_LAUNCH_REQUEST);
+ .getContainerToken(), NMCommunicatorEventType.CONTAINER_LAUNCH_REQUEST, launcherId);
this.clc = clc;
this.container = container;
+ this.taskCommId = taskCommId;
}
public ContainerLaunchContext getContainerLaunchContext() {
@@ -42,6 +45,10 @@ public class NMCommunicatorLaunchRequestEvent extends NMCommunicatorEvent {
return container;
}
+ public int getTaskCommId() {
+ return taskCommId;
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) {
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
index 277d1e7..c9b5c44 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/NMCommunicatorStopRequestEvent.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.yarn.api.records.Token;
public class NMCommunicatorStopRequestEvent extends NMCommunicatorEvent {
public NMCommunicatorStopRequestEvent(ContainerId containerId, NodeId nodeId,
- Token containerToken) {
+ Token containerToken, int launcherId) {
super(containerId, nodeId, containerToken,
- NMCommunicatorEventType.CONTAINER_STOP_REQUEST);
+ NMCommunicatorEventType.CONTAINER_STOP_REQUEST, launcherId);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index cb109ae..ae83730 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -108,9 +108,22 @@ public class TaskSchedulerEventHandler extends AbstractService
private final String[] taskSchedulerClasses;
protected final TaskSchedulerService []taskSchedulers;
+ private final boolean isPureLocalMode;
+ // If running in non local-only mode, the YARN task scheduler will always run to take care of
+ // registration with YARN and heartbeats to YARN.
+ // Splitting registration and heartbeats is not straigh-forward due to the taskScheduler being
+ // tied to a ContainerRequestType.
+ private final int yarnTaskSchedulerIndex;
+ // Custom AppIds to avoid container conflicts if there's multiple sources
+ private final long SCHEDULER_APP_ID_BASE = 111101111;
+ private final long SCHEDULER_APP_ID_INCREMENT = 111111111;
+
BlockingQueue<AMSchedulerEvent> eventQueue
= new LinkedBlockingQueue<AMSchedulerEvent>();
+ // Not tracking container / task to schedulerId. Instead relying on everything flowing through
+ // the system and being propagated back via events.
+
/**
*
* @param appContext
@@ -125,7 +138,7 @@ public class TaskSchedulerEventHandler extends AbstractService
public TaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
- String [] schedulerClasses) {
+ String [] schedulerClasses, boolean isPureLocalMode) {
super(TaskSchedulerEventHandler.class.getName());
this.appContext = appContext;
this.eventHandler = eventHandler;
@@ -133,13 +146,39 @@ public class TaskSchedulerEventHandler extends AbstractService
this.containerSignatureMatcher = containerSignatureMatcher;
this.webUI = webUI;
this.historyUrl = getHistoryUrl();
+ this.isPureLocalMode = isPureLocalMode;
if (this.webUI != null) {
this.webUI.setHistoryUrl(this.historyUrl);
}
- if (schedulerClasses == null || schedulerClasses.length == 0) {
- this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+
+ // Override everything for pure local mode
+ if (isPureLocalMode) {
+ this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT};
+ this.yarnTaskSchedulerIndex = -1;
} else {
- this.taskSchedulerClasses = schedulerClasses;
+ if (schedulerClasses == null || schedulerClasses.length ==0) {
+ this.taskSchedulerClasses = new String[] {TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT};
+ this.yarnTaskSchedulerIndex = 0;
+ } else {
+ // Ensure the YarnScheduler will be setup and note it's index. This will be responsible for heartbeats and YARN registration.
+ int foundYarnTaskSchedulerIndex = -1;
+ for (int i = 0 ; i < schedulerClasses.length ; i++) {
+ if (schedulerClasses[i].equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
+ foundYarnTaskSchedulerIndex = i;
+ break;
+ }
+ }
+ if (foundYarnTaskSchedulerIndex == -1) { // Not found. Add at the end.
+ this.taskSchedulerClasses = new String[schedulerClasses.length+1];
+ foundYarnTaskSchedulerIndex = this.taskSchedulerClasses.length -1;
+ for (int i = 0 ; i < schedulerClasses.length ; i++) { // Copy over the rest.
+ this.taskSchedulerClasses[i] = schedulerClasses[i];
+ }
+ } else {
+ this.taskSchedulerClasses = schedulerClasses;
+ }
+ this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
+ }
}
taskSchedulers = new TaskSchedulerService[this.taskSchedulerClasses.length];
}
@@ -157,12 +196,12 @@ public class TaskSchedulerEventHandler extends AbstractService
return cachedNodeCount;
}
- public Resource getAvailableResources() {
- return taskSchedulers[0].getAvailableResources();
+ public Resource getAvailableResources(int schedulerId) {
+ return taskSchedulers[schedulerId].getAvailableResources();
}
- public Resource getTotalResources() {
- return taskSchedulers[0].getTotalResources();
+ public Resource getTotalResources(int schedulerId) {
+ return taskSchedulers[schedulerId].getTotalResources();
}
public synchronized void handleEvent(AMSchedulerEvent sEvent) {
@@ -176,7 +215,7 @@ public class TaskSchedulerEventHandler extends AbstractService
switch(event.getState()) {
case FAILED:
case KILLED:
- handleTAUnsuccessfulEnd((AMSchedulerEventTAEnded) sEvent);
+ handleTAUnsuccessfulEnd(event);
break;
case SUCCEEDED:
handleTASucceeded(event);
@@ -228,9 +267,9 @@ public class TaskSchedulerEventHandler extends AbstractService
private void handleNodeBlacklistUpdate(AMSchedulerEventNodeBlacklistUpdate event) {
if (event.getType() == AMSchedulerEventType.S_NODE_BLACKLISTED) {
- taskSchedulers[0].blacklistNode(event.getNodeId());
+ taskSchedulers[event.getSchedulerId()].blacklistNode(event.getNodeId());
} else if (event.getType() == AMSchedulerEventType.S_NODE_UNBLACKLISTED) {
- taskSchedulers[0].unblacklistNode(event.getNodeId());
+ taskSchedulers[event.getSchedulerId()].unblacklistNode(event.getNodeId());
} else {
throw new TezUncheckedException("Invalid event type: " + event.getType());
}
@@ -242,14 +281,14 @@ public class TaskSchedulerEventHandler extends AbstractService
// TODO what happens to the task that was connected to this container?
// current assumption is that it will eventually call handleTaStopRequest
//TaskAttempt taskAttempt = (TaskAttempt)
- taskSchedulers[0].deallocateContainer(containerId);
+ taskSchedulers[event.getSchedulerId()].deallocateContainer(containerId);
// TODO does this container need to be stopped via C_STOP_REQUEST
sendEvent(new AMContainerEventStopRequest(containerId));
}
private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) {
TaskAttempt attempt = event.getAttempt();
- boolean wasContainerAllocated = taskSchedulers[0].deallocateTask(attempt, false);
+ boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt, false);
// use stored value of container id in case the scheduler has removed this
// assignment because the task has been deallocated earlier.
// retroactive case
@@ -291,7 +330,8 @@ public class TaskSchedulerEventHandler extends AbstractService
event.getAttemptID()));
}
- boolean wasContainerAllocated = taskSchedulers[0].deallocateTask(attempt, true);
+ boolean wasContainerAllocated = taskSchedulers[event.getSchedulerId()].deallocateTask(attempt,
+ true);
if (!wasContainerAllocated) {
LOG.error("De-allocated successful task: " + attempt.getID()
+ ", but TaskScheduler reported no container assigned to task");
@@ -316,7 +356,7 @@ public class TaskSchedulerEventHandler extends AbstractService
TaskAttempt affinityAttempt = vertex.getTask(taskIndex).getSuccessfulAttempt();
if (affinityAttempt != null) {
Preconditions.checkNotNull(affinityAttempt.getAssignedContainerID(), affinityAttempt.getID());
- taskSchedulers[0].allocateTask(taskAttempt,
+ taskSchedulers[event.getSchedulerId()].allocateTask(taskAttempt,
event.getCapability(),
affinityAttempt.getAssignedContainerID(),
Priority.newInstance(event.getPriority()),
@@ -336,7 +376,7 @@ public class TaskSchedulerEventHandler extends AbstractService
}
}
- taskSchedulers[0].allocateTask(taskAttempt,
+ taskSchedulers[event.getSchedulerId()].allocateTask(taskAttempt,
event.getCapability(),
hosts,
racks,
@@ -347,7 +387,8 @@ public class TaskSchedulerEventHandler extends AbstractService
private TaskSchedulerService createTaskScheduler(String host, int port, String trackingUrl,
AppContext appContext,
- String schedulerClassName) {
+ String schedulerClassName,
+ long customAppIdIdentifier) {
if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
@@ -355,7 +396,7 @@ public class TaskSchedulerEventHandler extends AbstractService
} else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Creating TaskScheduler: Local TaskScheduler");
return new LocalTaskSchedulerService(this, this.containerSignatureMatcher,
- host, port, trackingUrl, appContext);
+ host, port, trackingUrl, customAppIdIdentifier, appContext);
} else {
LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
// TODO TEZ-2003 Temporary reflection with specific parameters. Remove once there is a clean interface.
@@ -364,9 +405,10 @@ public class TaskSchedulerEventHandler extends AbstractService
try {
Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
.getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
- int.class, String.class, Configuration.class);
+ int.class, String.class, long.class, Configuration.class);
ctor.setAccessible(true);
- return ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
+ return ctor.newInstance(this, appContext, host, port, trackingUrl, customAppIdIdentifier,
+ getConfig());
} catch (NoSuchMethodException e) {
throw new TezUncheckedException(e);
} catch (InvocationTargetException e) {
@@ -381,10 +423,19 @@ public class TaskSchedulerEventHandler extends AbstractService
@VisibleForTesting
protected void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
+ // TODO Add error checking for components being used in the Vertex when running in pure local mode.
// Iterate over the list and create all the taskSchedulers
+ int j = 0;
for (int i = 0; i < taskSchedulerClasses.length; i++) {
+ long customAppIdIdentifier;
+ if (isPureLocalMode || taskSchedulerClasses[i].equals(
+ TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) { // Use the app identifier from the appId.
+ customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp();
+ } else {
+ customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT);
+ }
taskSchedulers[i] = createTaskScheduler(host, port,
- trackingUrl, appContext, taskSchedulerClasses[i]);
+ trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier);
}
}
@@ -403,12 +454,12 @@ public class TaskSchedulerEventHandler extends AbstractService
for (int i = 0 ; i < taskSchedulers.length ; i++) {
taskSchedulers[i].init(getConfig());
taskSchedulers[i].start();
- }
-
- // TODO TEZ-2118 Start using multiple task schedulers
- if (shouldUnregisterFlag.get()) {
- // Flag may have been set earlier when task scheduler was not initialized
- taskSchedulers[0].setShouldUnregister();
+ if (shouldUnregisterFlag.get()) {
+ // Flag may have been set earlier when task scheduler was not initialized
+ // TODO TEZ-2003 Should setRegister / unregister be part of APIs when not YARN specific ?
+ // External services could need to talk to some other entity.
+ taskSchedulers[i].setShouldUnregister();
+ }
}
this.eventHandlingThread = new Thread("TaskSchedulerEventHandlerThread") {
@@ -457,8 +508,10 @@ public class TaskSchedulerEventHandler extends AbstractService
if (eventHandlingThread != null)
eventHandlingThread.interrupt();
}
- if (taskSchedulers[0] != null) {
- ((AbstractService)taskSchedulers[0]).stop();
+ for (int i = 0 ; i < taskSchedulers.length ; i++) {
+ if (taskSchedulers[i] != null) {
+ taskSchedulers[i].stop();
+ }
}
}
@@ -467,15 +520,18 @@ public class TaskSchedulerEventHandler extends AbstractService
public synchronized void taskAllocated(Object task,
Object appCookie,
Container container) {
+ AMSchedulerEventTALaunchRequest event =
+ (AMSchedulerEventTALaunchRequest) appCookie;
ContainerId containerId = container.getId();
- if (appContext.getAllContainers().addContainerIfNew(container)) {
+ if (appContext.getAllContainers()
+ .addContainerIfNew(container, event.getSchedulerId(), event.getLauncherId(),
+ event.getTaskCommId())) {
appContext.getNodeTracker().nodeSeen(container.getNodeId());
sendEvent(new AMNodeEventContainerAllocated(container
.getNodeId(), container.getId()));
}
- AMSchedulerEventTALaunchRequest event =
- (AMSchedulerEventTALaunchRequest) appCookie;
+
TaskAttempt taskAttempt = event.getTaskAttempt();
// TODO - perhaps check if the task still needs this container
// because the deallocateTask downcall may have raced with the
@@ -484,7 +540,7 @@ public class TaskSchedulerEventHandler extends AbstractService
if (appContext.getAllContainers().get(containerId).getState() == AMContainerState.ALLOCATED) {
sendEvent(new AMContainerEventLaunchRequest(containerId, taskAttempt.getVertexID(),
- event.getContainerContext()));
+ event.getContainerContext(), event.getLauncherId(), event.getTaskCommId()));
}
sendEvent(new DAGEventSchedulerUpdateTAAssigned(taskAttempt, container));
sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getID(),
@@ -603,6 +659,9 @@ public class TaskSchedulerEventHandler extends AbstractService
public float getProgress() {
// at this point allocate has been called and so node count must be available
// may change after YARN-1722
+ // This is a heartbeat in from the scheduler into the APP, and is being used to piggy-back and
+ // node updates from the cluster.
+ // TODO Handle this in TEZ-2124. Need a way to know which scheduler is calling in.
int nodeCount = taskSchedulers[0].getClusterNodeCount();
if (nodeCount != cachedNodeCount) {
cachedNodeCount = nodeCount;
@@ -618,7 +677,9 @@ public class TaskSchedulerEventHandler extends AbstractService
}
public void dagCompleted() {
- taskSchedulers[0].dagComplete();
+ for (int i = 0 ; i < taskSchedulers.length ; i++) {
+ taskSchedulers[i].dagComplete();
+ }
}
public void dagSubmitted() {
@@ -628,7 +689,10 @@ public class TaskSchedulerEventHandler extends AbstractService
@Override
public void preemptContainer(ContainerId containerId) {
- taskSchedulers[0].deallocateContainer(containerId);
+ // TODO Why is this making a call back into the scheduler, when the call is originating from there.
+ // An AMContainer instance should already exist if an attempt is being made to preempt it
+ AMContainer amContainer = appContext.getAllContainers().get(containerId);
+ taskSchedulers[amContainer.getTaskSchedulerIdentifier()].deallocateContainer(containerId);
// Inform the Containers about completion.
sendEvent(new AMContainerEventCompleted(containerId, ContainerExitStatus.INVALID,
"Container preempted internally", TaskAttemptTerminationCause.INTERNAL_PREEMPTION));
@@ -637,13 +701,24 @@ public class TaskSchedulerEventHandler extends AbstractService
public void setShouldUnregisterFlag() {
LOG.info("TaskScheduler notified that it should unregister from RM");
this.shouldUnregisterFlag.set(true);
- if (this.taskSchedulers[0] != null) {
- this.taskSchedulers[0].setShouldUnregister();
+ for (int i = 0 ; i < taskSchedulers.length ; i++) {
+ if (this.taskSchedulers[i] != null) {
+ // TODO TEZ-2003 registration required for all schedulers ?
+ this.taskSchedulers[i].setShouldUnregister();
+ }
}
}
public boolean hasUnregistered() {
- return this.taskSchedulers[0].hasUnregistered();
+ boolean result = true;
+ for (int i = 0 ; i < taskSchedulers.length ; i++) {
+ // TODO TEZ-2003 registration required for all schedulers ?
+ result |= this.taskSchedulers[i].hasUnregistered();
+ if (result == false) {
+ return result;
+ }
+ }
+ return result;
}
@VisibleForTesting
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
index 0fc2e12..6616896 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
@@ -34,4 +34,7 @@ public interface AMContainer extends EventHandler<AMContainerEvent>{
public List<TezTaskAttemptID> getAllTaskAttempts();
public TezTaskAttemptID getCurrentTaskAttempt();
+ public int getTaskSchedulerIdentifier();
+ public int getContainerLauncherIdentifier();
+ public int getTaskCommunicatorIdentifier();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java
index d973264..92e5817 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventLaunchRequest.java
@@ -27,12 +27,17 @@ public class AMContainerEventLaunchRequest extends AMContainerEvent {
private final TezVertexID vertexId;
private final ContainerContext containerContext;
+ private final int launcherId;
+ private final int taskCommId;
public AMContainerEventLaunchRequest(ContainerId containerId,
- TezVertexID vertexId, ContainerContext containerContext) {
+ TezVertexID vertexId, ContainerContext containerContext,
+ int launcherId, int taskCommId) {
super(containerId, AMContainerEventType.C_LAUNCH_REQUEST);
this.vertexId = vertexId;
this.containerContext = containerContext;
+ this.launcherId = launcherId;
+ this.taskCommId = taskCommId;
}
public TezDAGID getDAGId() {
@@ -46,4 +51,12 @@ public class AMContainerEventLaunchRequest extends AMContainerEvent {
public ContainerContext getContainerContext() {
return this.containerContext;
}
+
+ public int getLauncherId() {
+ return launcherId;
+ }
+
+ public int getTaskCommId() {
+ return taskCommId;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index 1acec9c..39df2e8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -81,6 +81,9 @@ public class AMContainerImpl implements AMContainer {
private final TaskAttemptListener taskAttemptListener;
protected final EventHandler eventHandler;
private final ContainerSignatureMatcher signatureMatcher;
+ private final int schedulerId;
+ private final int launcherId;
+ private final int taskCommId;
private final List<TezTaskAttemptID> completedAttempts =
new LinkedList<TezTaskAttemptID>();
@@ -302,7 +305,7 @@ public class AMContainerImpl implements AMContainer {
// additional change - JvmID, YarnChild, etc depend on TaskType.
public AMContainerImpl(Container container, ContainerHeartbeatHandler chh,
TaskAttemptListener tal, ContainerSignatureMatcher signatureMatcher,
- AppContext appContext) {
+ AppContext appContext, int schedulerId, int launcherId, int taskCommId) {
ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
this.readLock = rwLock.readLock();
this.writeLock = rwLock.writeLock();
@@ -314,6 +317,9 @@ public class AMContainerImpl implements AMContainer {
this.containerHeartbeatHandler = chh;
this.taskAttemptListener = tal;
this.failedAssignments = new LinkedList<TezTaskAttemptID>();
+ this.schedulerId = schedulerId;
+ this.launcherId = launcherId;
+ this.taskCommId = taskCommId;
this.stateMachine = stateMachineFactory.make(this);
}
@@ -363,6 +369,21 @@ public class AMContainerImpl implements AMContainer {
}
}
+ @Override
+ public int getTaskSchedulerIdentifier() {
+ return this.schedulerId;
+ }
+
+ @Override
+ public int getContainerLauncherIdentifier() {
+ return this.launcherId;
+ }
+
+ @Override
+ public int getTaskCommunicatorIdentifier() {
+ return this.taskCommId;
+ }
+
public boolean isInErrorState() {
return inError;
}
@@ -432,7 +453,7 @@ public class AMContainerImpl implements AMContainer {
containerContext.getLocalResources(),
containerContext.getEnvironment(),
containerContext.getJavaOpts(),
- container.taskAttemptListener.getAddress(), containerContext.getCredentials(),
+ container.taskAttemptListener.getTaskCommunicator(container.taskCommId).getAddress(), containerContext.getCredentials(),
container.appContext, container.container.getResource(),
container.appContext.getAMConf());
@@ -1014,7 +1035,7 @@ public class AMContainerImpl implements AMContainer {
}
protected void deAllocate() {
- sendEvent(new AMSchedulerEventDeallocateContainer(containerId));
+ sendEvent(new AMSchedulerEventDeallocateContainer(containerId, schedulerId));
}
protected void sendTerminatedToTaskAttempt(
@@ -1044,28 +1065,28 @@ public class AMContainerImpl implements AMContainer {
}
protected void sendStartRequestToNM(ContainerLaunchContext clc) {
- sendEvent(new NMCommunicatorLaunchRequestEvent(clc, container));
+ sendEvent(new NMCommunicatorLaunchRequestEvent(clc, container, launcherId, taskCommId));
}
protected void sendStopRequestToNM() {
sendEvent(new NMCommunicatorStopRequestEvent(containerId,
- container.getNodeId(), container.getContainerToken()));
+ container.getNodeId(), container.getContainerToken(), launcherId));
}
protected void unregisterAttemptFromListener(TezTaskAttemptID attemptId) {
- taskAttemptListener.unregisterTaskAttempt(attemptId);
+ taskAttemptListener.unregisterTaskAttempt(attemptId, taskCommId);
}
protected void registerAttemptWithListener(AMContainerTask amContainerTask) {
- taskAttemptListener.registerTaskAttempt(amContainerTask, this.containerId);
+ taskAttemptListener.registerTaskAttempt(amContainerTask, this.containerId, taskCommId);
}
protected void registerWithTAListener() {
- taskAttemptListener.registerRunningContainer(containerId);
+ taskAttemptListener.registerRunningContainer(containerId, taskCommId);
}
protected void unregisterFromTAListener() {
- this.taskAttemptListener.unregisterRunningContainer(containerId);
+ this.taskAttemptListener.unregisterRunningContainer(containerId, taskCommId);
}
protected void registerWithContainerListener() {
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
index 574c38e..938096d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
@@ -62,9 +62,9 @@ public class AMContainerMap extends AbstractService implements EventHandler<AMCo
}
}
- public boolean addContainerIfNew(Container container) {
+ public boolean addContainerIfNew(Container container, int schedulerId, int launcherId, int taskCommId) {
AMContainer amc = new AMContainerImpl(container, chh, tal,
- containerSignatureMatcher, context);
+ containerSignatureMatcher, context, schedulerId, launcherId, taskCommId);
return (containerMap.putIfAbsent(container.getId(), amc) == null);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
index b93cab3..0d8e4cd 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
@@ -257,7 +257,8 @@ public class AMNodeImpl implements AMNode {
// these containers are not useful anymore
pastContainers.addAll(containers);
containers.clear();
- sendEvent(new AMSchedulerEventNodeBlacklistUpdate(getNodeId(), true));
+ // TODO TEZ-2124 node tracking per ext source
+ sendEvent(new AMSchedulerEventNodeBlacklistUpdate(getNodeId(), true, 0));
}
@SuppressWarnings("unchecked")
@@ -363,7 +364,8 @@ public class AMNodeImpl implements AMNode {
public void transition(AMNodeImpl node, AMNodeEvent nEvent) {
node.ignoreBlacklisting = ignore;
if (node.getState() == AMNodeState.BLACKLISTED) {
- node.sendEvent(new AMSchedulerEventNodeBlacklistUpdate(node.getNodeId(), false));
+ // TODO TEZ-2124 node tracking per ext source
+ node.sendEvent(new AMSchedulerEventNodeBlacklistUpdate(node.getNodeId(), false, 0));
}
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 8763a0c..b70b9ea 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -198,7 +198,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
@Override
public void serviceStart() throws Exception {
taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
- taskCommunicator = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator();
+ taskCommunicator = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator(0);
eventHandlingThread = new Thread(this);
eventHandlingThread.start();
ExecutorService rawExecutor = Executors.newFixedThreadPool(handlerConcurrency,
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 219217f..200e737 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -125,11 +125,10 @@ public class TestTaskAttemptListenerImplTezDag {
doReturn(container).when(amContainer).getContainer();
taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null);
TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
-
taskSpec = mock(TaskSpec.class);
doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
amContainerTask = new AMContainerTask(taskSpec, null, null, false, 0);
@@ -139,7 +138,7 @@ public class TestTaskAttemptListenerImplTezDag {
@Test(timeout = 5000)
public void testGetTask() throws IOException {
- TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+ TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator(0);
TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
ContainerId containerId1 = createContainerId(appId, 1);
@@ -149,55 +148,55 @@ public class TestTaskAttemptListenerImplTezDag {
ContainerId containerId2 = createContainerId(appId, 2);
ContainerContext containerContext2 = new ContainerContext(containerId2.toString());
- taskAttemptListener.registerRunningContainer(containerId2);
+ taskAttemptListener.registerRunningContainer(containerId2, 0);
containerTask = tezUmbilical.getTask(containerContext2);
assertNull(containerTask);
// Valid task registered
- taskAttemptListener.registerTaskAttempt(amContainerTask, containerId2);
+ taskAttemptListener.registerTaskAttempt(amContainerTask, containerId2, 0);
containerTask = tezUmbilical.getTask(containerContext2);
assertFalse(containerTask.shouldDie());
assertEquals(taskSpec, containerTask.getTaskSpec());
// Task unregistered. Should respond to heartbeats
- taskAttemptListener.unregisterTaskAttempt(taskAttemptId);
+ taskAttemptListener.unregisterTaskAttempt(taskAttemptId, 0);
containerTask = tezUmbilical.getTask(containerContext2);
assertNull(containerTask);
// Container unregistered. Should send a shouldDie = true
- taskAttemptListener.unregisterRunningContainer(containerId2);
+ taskAttemptListener.unregisterRunningContainer(containerId2, 0);
containerTask = tezUmbilical.getTask(containerContext2);
assertTrue(containerTask.shouldDie());
ContainerId containerId3 = createContainerId(appId, 3);
ContainerContext containerContext3 = new ContainerContext(containerId3.toString());
- taskAttemptListener.registerRunningContainer(containerId3);
+ taskAttemptListener.registerRunningContainer(containerId3, 0);
// Register task to container3, followed by unregistering container 3 all together
TaskSpec taskSpec2 = mock(TaskSpec.class);
TezTaskAttemptID taskAttemptId2 = mock(TezTaskAttemptID.class);
doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false, 0);
- taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3);
- taskAttemptListener.unregisterRunningContainer(containerId3);
+ taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3, 0);
+ taskAttemptListener.unregisterRunningContainer(containerId3, 0);
containerTask = tezUmbilical.getTask(containerContext3);
assertTrue(containerTask.shouldDie());
}
@Test(timeout = 5000)
public void testGetTaskMultiplePulls() throws IOException {
- TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+ TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator(0);
TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
ContainerId containerId1 = createContainerId(appId, 1);
doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
- taskAttemptListener.registerRunningContainer(containerId1);
+ taskAttemptListener.registerRunningContainer(containerId1, 0);
containerTask = tezUmbilical.getTask(containerContext1);
assertNull(containerTask);
// Register task
- taskAttemptListener.registerTaskAttempt(amContainerTask, containerId1);
+ taskAttemptListener.registerTaskAttempt(amContainerTask, containerId1, 0);
containerTask = tezUmbilical.getTask(containerContext1);
assertFalse(containerTask.shouldDie());
assertEquals(taskSpec, containerTask.getTaskSpec());
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
index d21f715..d6fc46e 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.tez.common.MockDNSToSwitchMapping;
+import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
@@ -285,8 +286,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = new MockEventHandler();
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -335,8 +337,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = spy(new MockEventHandler());
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -357,7 +360,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
@@ -436,8 +439,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = new MockEventHandler();
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -458,7 +462,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
@@ -501,8 +505,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = spy(new MockEventHandler());
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -523,7 +528,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
@@ -593,8 +598,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = spy(new MockEventHandler());
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -616,7 +622,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
@@ -724,8 +730,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = spy(new MockEventHandler());
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -747,7 +754,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
@@ -816,8 +823,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = spy(new MockEventHandler());
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -838,7 +846,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
@@ -911,8 +919,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = spy(new MockEventHandler());
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -933,7 +942,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
@@ -1014,8 +1023,9 @@ public class TestTaskAttempt {
MockEventHandler eventHandler = spy(new MockEventHandler());
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1036,7 +1046,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
@@ -1114,8 +1124,9 @@ public class TestTaskAttempt {
MockEventHandler mockEh = new MockEventHandler();
MockEventHandler eventHandler = spy(mockEh);
TaskAttemptListener taListener = mock(TaskAttemptListener.class);
- when(taListener.getAddress()).thenReturn(
- new InetSocketAddress("localhost", 0));
+ TaskCommunicator taskComm = mock(TaskCommunicator.class);
+ doReturn(new InetSocketAddress("localhost", 0)).when(taskComm).getAddress();
+ doReturn(taskComm).when(taListener).getTaskCommunicator(0);
Configuration taskConf = new Configuration();
taskConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -1136,7 +1147,7 @@ public class TestTaskAttempt {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appCtx);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
doReturn(containers).when(appCtx).getAllContainers();
http://git-wip-us.apache.org/repos/asf/tez/blob/359eba29/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
index 8864e9f..6ee741a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
@@ -2196,7 +2196,7 @@ public class TestVertexImpl {
doReturn(dagId).when(appContext).getCurrentDAGID();
doReturn(dagId).when(dag).getID();
doReturn(taskScheduler).when(appContext).getTaskScheduler();
- doReturn(Resource.newInstance(102400, 60)).when(taskScheduler).getTotalResources();
+ doReturn(Resource.newInstance(102400, 60)).when(taskScheduler).getTotalResources(0);
doReturn(historyEventHandler).when(appContext).getHistoryHandler();
doReturn(dispatcher.getEventHandler()).when(appContext).getEventHandler();
doReturn(clock).when(appContext).getClock();
@@ -3229,7 +3229,7 @@ public class TestVertexImpl {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appContext);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(containers).when(appContext).getAllContainers();
ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));
@@ -3264,7 +3264,7 @@ public class TestVertexImpl {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appContext);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(containers).when(appContext).getAllContainers();
ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));
@@ -3300,7 +3300,7 @@ public class TestVertexImpl {
AMContainerMap containers = new AMContainerMap(
mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
new ContainerContextMatcher(), appContext);
- containers.addContainerIfNew(container);
+ containers.addContainerIfNew(container, 0, 0, 0);
doReturn(containers).when(appContext).getAllContainers();
ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));
[48/51] [abbrv] tez git commit: TEZ-2441. Add tests for
TezTaskRunner2. (sseth)
Posted by ss...@apache.org.
TEZ-2441. Add tests for TezTaskRunner2. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/eb82ca2c
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/eb82ca2c
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/eb82ca2c
Branch: refs/heads/TEZ-2003
Commit: eb82ca2c6af9c3e1b10eb372398fd6f2530508c1
Parents: 80e7053
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Jul 29 18:25:18 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:58 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/runtime/task/TezChild.java | 5 +-
.../apache/tez/runtime/task/TezTaskRunner.java | 2 +-
.../apache/tez/runtime/task/TezTaskRunner2.java | 42 +-
.../runtime/task/TaskExecutionTestHelpers.java | 451 +++++++++++++
.../runtime/task/TestContainerExecution.java | 59 ++
.../tez/runtime/task/TestTaskExecution.java | 400 +-----------
.../tez/runtime/task/TestTaskExecution2.java | 638 +++++++++++++++++++
.../src/test/resources/log4j.properties | 19 +
9 files changed, 1213 insertions(+), 404 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index b88044b..9d72d92 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -39,5 +39,6 @@ ALL CHANGES:
TEZ-2651. Pluggable services should not extend AbstractService.
TEZ-2652. Cleanup the way services are specified for an AM and vertices.
TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
+ TEZ-2441. Add tests for TezTaskRunner2.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index 353fe23..b64ec37 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -49,7 +49,6 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
-import org.apache.log4j.LogManager;
import org.apache.tez.common.ContainerContext;
import org.apache.tez.common.ContainerTask;
import org.apache.tez.common.TezCommonUtils;
@@ -68,7 +67,6 @@ import org.apache.tez.dag.utils.RelocalizationUtils;
import org.apache.tez.runtime.api.ExecutionContext;
import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
import org.apache.tez.runtime.api.impl.TaskSpec;
-import org.apache.tez.runtime.api.impl.TezUmbilical;
import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
import org.apache.tez.runtime.internals.api.TaskReporterInterface;
import org.slf4j.Logger;
@@ -256,6 +254,7 @@ public class TezChild {
boolean shouldDie;
try {
TaskRunner2Result result = taskRunner.run();
+ LOG.info("TaskRunner2Result: {}", result);
shouldDie = result.isContainerShutdownRequested();
if (shouldDie) {
LOG.info("Got a shouldDie notification via heartbeats for container {}. Shutting down", containerIdString);
@@ -377,8 +376,6 @@ public class TezChild {
}
if (ownUmbilical) {
RPC.stopProxy(umbilical);
- // TODO Temporary change. Revert. Ideally, move this over to the main method in TezChild if possible.
-// LogManager.shutdown();
}
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index a82d87b..aebf6a9 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -250,7 +250,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
cause = ((UndeclaredThrowableException) cause).getCause();
}
maybeRegisterFirstException(cause);
- LOG.info("Encounted an error while executing task: " + task.getTaskAttemptID(),
+ LOG.info("Encountered an error while executing task: " + task.getTaskAttemptID(),
cause);
try {
sendFailure(cause, "Failure while running task");
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
index a5fabb5..1a8828d 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner2.java
@@ -24,6 +24,7 @@ import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Multimap;
import com.google.common.util.concurrent.ListenableFuture;
@@ -48,6 +49,9 @@ import org.slf4j.LoggerFactory;
public class TezTaskRunner2 {
+ // Behaviour changes as compared to TezTaskRunner
+ // - Exception not thrown. Instead returned in the result.
+ // - The actual exception is part of the result, instead of requiring a getCause().
private static final Logger LOG = LoggerFactory.getLogger(TezTaskRunner2.class);
@@ -156,19 +160,7 @@ public class TezTaskRunner2 {
}
}
}
- if (executionResult != null) {
- synchronized (this) {
- if (isRunningState()) {
- if (executionResult.error != null) {
- trySettingEndReason(EndReason.TASK_ERROR);
- registerFirstException(executionResult.error, null);
- } else {
- trySettingEndReason(EndReason.SUCCESS);
- taskComplete.set(true);
- }
- }
- }
- }
+ processCallableResult(executionResult);
switch (firstEndReason) {
case SUCCESS:
@@ -249,6 +241,26 @@ public class TezTaskRunner2 {
}
}
+ // It's possible for the task to actually complete, and an alternate signal such as killTask/killContainer
+ // come in before the future has been processed by this thread. That condition is not handled - and
+ // the result of the execution will be determind by the thread order.
+ @VisibleForTesting
+ void processCallableResult(TaskRunner2CallableResult executionResult) {
+ if (executionResult != null) {
+ synchronized (this) {
+ if (isRunningState()) {
+ if (executionResult.error != null) {
+ trySettingEndReason(EndReason.TASK_ERROR);
+ registerFirstException(executionResult.error, null);
+ } else {
+ trySettingEndReason(EndReason.SUCCESS);
+ taskComplete.set(true);
+ }
+ }
+ }
+ }
+ }
+
/**
* Attempt to kill the running task, if it hasn't already completed for some other reason.
* @return true if the task kill was honored, false otherwise
@@ -438,12 +450,12 @@ public class TezTaskRunner2 {
private String getTaskDiagnosticsString(Throwable t, String message) {
String diagnostics;
if (t != null && message != null) {
- diagnostics = "exceptionThrown=" + ExceptionUtils.getStackTrace(t) + ", errorMessage="
+ diagnostics = "Failure while running task: " + ExceptionUtils.getStackTrace(t) + ", errorMessage="
+ message;
} else if (t == null && message == null) {
diagnostics = "Unknown error";
} else {
- diagnostics = t != null ? "exceptionThrown=" + ExceptionUtils.getStackTrace(t)
+ diagnostics = t != null ? "Failure while running task: " + ExceptionUtils.getStackTrace(t)
: " errorMessage=" + message;
}
return diagnostics;
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
new file mode 100644
index 0000000..fc42da3
--- /dev/null
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
@@ -0,0 +1,451 @@
+/*
+ * 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.tez.runtime.task;
+
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.AbstractLogicalIOProcessor;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.ProcessorContext;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TaskExecutionTestHelpers {
+
+ public static final String HEARTBEAT_EXCEPTION_STRING = "HeartbeatException";
+
+ // Uses static fields for signaling. Ensure only used by one test at a time.
+ public static class TestProcessor extends AbstractLogicalIOProcessor {
+
+ public static final byte[] CONF_EMPTY = new byte[] { 0 };
+ public static final byte[] CONF_THROW_IO_EXCEPTION = new byte[] { 1 };
+ public static final byte[] CONF_THROW_TEZ_EXCEPTION = new byte[] { 2 };
+ public static final byte[] CONF_SIGNAL_FATAL_AND_THROW = new byte[] { 4 };
+ public static final byte[] CONF_SIGNAL_FATAL_AND_LOOP = new byte[] { 8 };
+ public static final byte[] CONF_SIGNAL_FATAL_AND_COMPLETE = new byte[] { 16 };
+
+ private static final Logger LOG = LoggerFactory.getLogger(TestProcessor.class);
+
+ private static final ReentrantLock processorLock = new ReentrantLock();
+ private static final Condition processorCondition = processorLock.newCondition();
+ private static final Condition loopCondition = processorLock.newCondition();
+ private static final Condition completionCondition = processorLock.newCondition();
+ private static final Condition runningCondition = processorLock.newCondition();
+ private static volatile boolean completed = false;
+ private static volatile boolean running = false;
+ private static volatile boolean looping = false;
+ private static volatile boolean signalled = false;
+
+ private static boolean receivedInterrupt = false;
+ private static volatile boolean wasAborted = false;
+
+ private boolean throwIOException = false;
+ private boolean throwTezException = false;
+ private boolean signalFatalAndThrow = false;
+ private boolean signalFatalAndLoop = false;
+ private boolean signalFatalAndComplete = false;
+
+ public TestProcessor(ProcessorContext context) {
+ super(context);
+ }
+
+ @Override
+ public void initialize() throws Exception {
+ parseConf(getContext().getUserPayload().deepCopyAsArray());
+ }
+
+ @Override
+ public void handleEvents(List<Event> processorEvents) {
+
+ }
+
+ @Override
+ public void close() throws Exception {
+
+ }
+
+ private void parseConf(byte[] bytes) {
+ byte b = bytes[0];
+ throwIOException = (b & 1) > 0;
+ throwTezException = (b & 2) > 0;
+ signalFatalAndThrow = (b & 4) > 0;
+ signalFatalAndLoop = (b & 8) > 0;
+ signalFatalAndComplete = (b & 16) > 0;
+ }
+
+ public static void reset() {
+ signalled = false;
+ receivedInterrupt = false;
+ completed = false;
+ running = false;
+ wasAborted = false;
+ }
+
+ public static void signal() {
+ LOG.info("Signalled");
+ processorLock.lock();
+ try {
+ signalled = true;
+ processorCondition.signal();
+ } finally {
+ processorLock.unlock();
+ }
+ }
+
+ public static void awaitStart() throws InterruptedException {
+ LOG.info("Awaiting Process run");
+ processorLock.lock();
+ try {
+ if (running) {
+ return;
+ }
+ runningCondition.await();
+ } finally {
+ processorLock.unlock();
+ }
+ }
+
+ public static void awaitLoop() throws InterruptedException {
+ LOG.info("Awaiting loop after signalling error");
+ processorLock.lock();
+ try {
+ if (looping) {
+ return;
+ }
+ loopCondition.await();
+ } finally {
+ processorLock.unlock();
+ }
+ }
+
+ public static void awaitCompletion() throws InterruptedException {
+ LOG.info("Await completion");
+ processorLock.lock();
+ try {
+ if (completed) {
+ return;
+ } else {
+ completionCondition.await();
+ }
+ } finally {
+ processorLock.unlock();
+ }
+ }
+
+ public static boolean wasInterrupted() {
+ processorLock.lock();
+ try {
+ return receivedInterrupt;
+ } finally {
+ processorLock.unlock();
+ }
+ }
+
+ public static boolean wasAborted() {
+ processorLock.lock();
+ try {
+ return wasAborted;
+ } finally {
+ processorLock.unlock();
+ }
+ }
+
+ @Override
+ public void abort() {
+ wasAborted = true;
+ }
+
+ @Override
+ public void run(Map<String, LogicalInput> inputs, Map<String, LogicalOutput> outputs) throws
+ Exception {
+ processorLock.lock();
+ running = true;
+ runningCondition.signal();
+ try {
+ try {
+ LOG.info("Signal is: " + signalled);
+ if (!signalled) {
+ LOG.info("Waiting for processor signal");
+ processorCondition.await();
+ }
+ if (Thread.currentThread().isInterrupted()) {
+ throw new InterruptedException();
+ }
+ LOG.info("Received processor signal");
+ if (throwIOException) {
+ throw createProcessorIOException();
+ } else if (throwTezException) {
+ throw createProcessorTezException();
+ } else if (signalFatalAndThrow) {
+ IOException io = new IOException("FATALERROR");
+ getContext().fatalError(io, "FATALERROR");
+ throw io;
+ } else if (signalFatalAndComplete) {
+ IOException io = new IOException("FATALERROR");
+ getContext().fatalError(io, "FATALERROR");
+ return;
+ } else if (signalFatalAndLoop) {
+ IOException io = createProcessorIOException();
+ getContext().fatalError(io, "FATALERROR");
+ LOG.info("looping");
+ looping = true;
+ loopCondition.signal();
+ LOG.info("Waiting for Processor signal again");
+ processorCondition.await();
+ LOG.info("Received second processor signal");
+ }
+ } catch (InterruptedException e) {
+ receivedInterrupt = true;
+ }
+ } finally {
+ completed = true;
+ completionCondition.signal();
+ processorLock.unlock();
+ }
+ }
+ }
+
+ public static TezException createProcessorTezException() {
+ return new TezException("TezException");
+ }
+
+ public static IOException createProcessorIOException() {
+ return new IOException("IOException");
+ }
+
+ public static class TezTaskUmbilicalForTest implements TezTaskUmbilicalProtocol {
+
+ private static final Logger LOG = LoggerFactory.getLogger(TezTaskUmbilicalForTest.class);
+
+ private final List<TezEvent> requestEvents = new LinkedList<TezEvent>();
+
+ private final ReentrantLock umbilicalLock = new ReentrantLock();
+ private final Condition eventCondition = umbilicalLock.newCondition();
+ private boolean pendingEvent = false;
+ private boolean eventEnacted = false;
+
+ volatile int getTaskInvocations = 0;
+
+ private boolean shouldThrowException = false;
+ private boolean shouldSendDieSignal = false;
+
+ public void signalThrowException() {
+ umbilicalLock.lock();
+ try {
+ shouldThrowException = true;
+ pendingEvent = true;
+ } finally {
+ umbilicalLock.unlock();
+ }
+ }
+
+ public void signalSendShouldDie() {
+ umbilicalLock.lock();
+ try {
+ shouldSendDieSignal = true;
+ pendingEvent = true;
+ } finally {
+ umbilicalLock.unlock();
+ }
+ }
+
+ public void awaitRegisteredEvent() throws InterruptedException {
+ umbilicalLock.lock();
+ try {
+ if (eventEnacted) {
+ return;
+ }
+ LOG.info("Awaiting event");
+ eventCondition.await();
+ } finally {
+ umbilicalLock.unlock();
+ }
+ }
+
+ public void resetTrackedEvents() {
+ umbilicalLock.lock();
+ try {
+ requestEvents.clear();
+ } finally {
+ umbilicalLock.unlock();
+ }
+ }
+
+ public void verifyNoCompletionEvents() {
+ umbilicalLock.lock();
+ try {
+ for (TezEvent event : requestEvents) {
+ if (event.getEvent() instanceof TaskAttemptFailedEvent) {
+ fail("Found a TaskAttemptFailedEvent when not expected");
+ }
+ if (event.getEvent() instanceof TaskAttemptCompletedEvent) {
+ fail("Found a TaskAttemptCompletedvent when not expected");
+ }
+ }
+ } finally {
+ umbilicalLock.unlock();
+ }
+ }
+
+ public void verifyTaskFailedEvent(String diagnostics) {
+ umbilicalLock.lock();
+ try {
+ for (TezEvent event : requestEvents) {
+ if (event.getEvent() instanceof TaskAttemptFailedEvent) {
+ TaskAttemptFailedEvent failedEvent = (TaskAttemptFailedEvent) event.getEvent();
+ if (failedEvent.getDiagnostics().startsWith(diagnostics)) {
+ return;
+ } else {
+ fail("Diagnostic message does not match expected message. Found [" +
+ failedEvent.getDiagnostics() + "], Expected: [" + diagnostics + "]");
+ }
+ }
+ }
+ fail("No TaskAttemptFailedEvents sent over umbilical");
+ } finally {
+ umbilicalLock.unlock();
+ }
+ }
+
+ public void verifyTaskFailedEvent(String diagStart, String diagContains) {
+ umbilicalLock.lock();
+ try {
+ for (TezEvent event : requestEvents) {
+ if (event.getEvent() instanceof TaskAttemptFailedEvent) {
+ TaskAttemptFailedEvent failedEvent = (TaskAttemptFailedEvent) event.getEvent();
+ if (failedEvent.getDiagnostics().startsWith(diagStart)) {
+ if (diagContains != null) {
+ if (failedEvent.getDiagnostics().contains(diagContains)) {
+ return;
+ } else {
+ fail("Diagnostic message does not contain expected message. Found [" +
+ failedEvent.getDiagnostics() + "], Expected: [" + diagContains + "]");
+ }
+ }
+ } else {
+ fail("Diagnostic message does not start with expected message. Found [" +
+ failedEvent.getDiagnostics() + "], Expected: [" + diagStart + "]");
+ }
+ }
+ }
+ fail("No TaskAttemptFailedEvents sent over umbilical");
+ } finally {
+ umbilicalLock.unlock();
+ }
+ }
+
+ public void verifyTaskSuccessEvent() {
+ umbilicalLock.lock();
+ try {
+ for (TezEvent event : requestEvents) {
+ if (event.getEvent() instanceof TaskAttemptCompletedEvent) {
+ return;
+ }
+ }
+ fail("No TaskAttemptFailedEvents sent over umbilical");
+ } finally {
+ umbilicalLock.unlock();
+ }
+ }
+
+ @Override
+ public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
+ return 0;
+ }
+
+ @Override
+ public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
+ int clientMethodsHash) throws IOException {
+ return null;
+ }
+
+ @Override
+ public ContainerTask getTask(ContainerContext containerContext) throws IOException {
+ // Return shouldDie = true
+ getTaskInvocations++;
+ return new ContainerTask(null, true, null, null, false);
+ }
+
+ @Override
+ public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
+ return true;
+ }
+
+ @Override
+ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException,
+ TezException {
+ umbilicalLock.lock();
+ if (request.getEvents() != null) {
+ requestEvents.addAll(request.getEvents());
+ }
+ try {
+ if (shouldThrowException) {
+ LOG.info("TestUmbilical throwing Exception");
+ throw new IOException(HEARTBEAT_EXCEPTION_STRING);
+ }
+ TezHeartbeatResponse response = new TezHeartbeatResponse();
+ response.setLastRequestId(request.getRequestId());
+ if (shouldSendDieSignal) {
+ LOG.info("TestUmbilical returning shouldDie=true");
+ response.setShouldDie();
+ }
+ return response;
+ } finally {
+ if (pendingEvent) {
+ eventEnacted = true;
+ LOG.info("Signalling Event");
+ eventCondition.signal();
+ }
+ umbilicalLock.unlock();
+ }
+ }
+ }
+
+ public static ContainerId createContainerId(ApplicationId appId) {
+ ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+ ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+ return containerId;
+ }
+
+ public static TaskReporter createTaskReporter(ApplicationId appId, TezTaskUmbilicalForTest umbilical) {
+ TaskReporter taskReporter = new TaskReporter(umbilical, 100, 1000, 100, new AtomicLong(0),
+ createContainerId(appId).toString());
+ return taskReporter;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
new file mode 100644
index 0000000..c1616af
--- /dev/null
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestContainerExecution.java
@@ -0,0 +1,59 @@
+/*
+ * 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.tez.runtime.task;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.junit.Test;
+
+public class TestContainerExecution {
+
+ @Test(timeout = 5000)
+ public void testGetTaskShouldDie() throws InterruptedException, ExecutionException {
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+ ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ ContainerContext containerContext = new ContainerContext(containerId.toString());
+
+ ContainerReporter containerReporter = new ContainerReporter(umbilical, containerContext, 100);
+ ListenableFuture<ContainerTask> getTaskFuture = executor.submit(containerReporter);
+
+ getTaskFuture.get();
+ assertEquals(1, umbilical.getTaskInvocations);
+
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java
index 1bcb337..a99416a 100644
--- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution.java
@@ -18,8 +18,8 @@
package org.apache.tez.runtime.task;
+import static org.apache.tez.runtime.task.TaskExecutionTestHelpers.createTaskReporter;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -28,30 +28,18 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
import org.apache.tez.dag.api.ProcessorDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezException;
@@ -60,21 +48,13 @@ import org.apache.tez.dag.records.TezDAGID;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezTaskID;
import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.runtime.api.AbstractLogicalIOProcessor;
-import org.apache.tez.runtime.api.Event;
-import org.apache.tez.runtime.api.LogicalInput;
-import org.apache.tez.runtime.api.LogicalOutput;
-import org.apache.tez.runtime.api.ProcessorContext;
-import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
-import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
import org.apache.tez.runtime.api.impl.InputSpec;
import org.apache.tez.runtime.api.impl.OutputSpec;
import org.apache.tez.runtime.api.impl.TaskSpec;
-import org.apache.tez.runtime.api.impl.TezEvent;
-import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
-import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
import org.apache.tez.runtime.common.resources.ScalingAllocator;
+import org.apache.tez.runtime.task.TaskExecutionTestHelpers.TestProcessor;
+import org.apache.tez.runtime.task.TaskExecutionTestHelpers.TezTaskUmbilicalForTest;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.Test;
@@ -82,7 +62,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.HashMultimap;
-import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
@@ -91,7 +70,7 @@ public class TestTaskExecution {
private static final Logger LOG = LoggerFactory.getLogger(TestTaskExecution.class);
- private static final String HEARTBEAT_EXCEPTION_STRING = "HeartbeatException";
+
private static final Configuration defaultConf = new Configuration();
private static final FileSystem localFs;
@@ -137,7 +116,7 @@ public class TestTaskExecution {
TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
TestProcessor.CONF_EMPTY);
// Setup the executor
- Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+ Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
// Signal the processor to go through
TestProcessor.signal();
boolean result = taskRunnerFuture.get();
@@ -164,7 +143,7 @@ public class TestTaskExecution {
TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
TestProcessor.CONF_EMPTY);
// Setup the executor
- Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+ Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
// Signal the processor to go through
TestProcessor.signal();
boolean result = taskRunnerFuture.get();
@@ -176,7 +155,7 @@ public class TestTaskExecution {
taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
TestProcessor.CONF_EMPTY);
// Setup the executor
- taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+ taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
// Signal the processor to go through
TestProcessor.signal();
result = taskRunnerFuture.get();
@@ -188,7 +167,7 @@ public class TestTaskExecution {
}
}
- // test tasked failed due to exception in Processor
+ // test task failed due to exception in Processor
@Test(timeout = 5000)
public void testFailedTask() throws IOException, InterruptedException, TezException {
@@ -203,7 +182,7 @@ public class TestTaskExecution {
TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
TestProcessor.CONF_THROW_TEZ_EXCEPTION);
// Setup the executor
- Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+ Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
// Signal the processor to go through
TestProcessor.awaitStart();
TestProcessor.signal();
@@ -238,7 +217,7 @@ public class TestTaskExecution {
TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
"NotExitedProcessor", TestProcessor.CONF_THROW_TEZ_EXCEPTION);
// Setup the executor
- Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+ Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
try {
taskRunnerFuture.get();
} catch (ExecutionException e) {
@@ -268,7 +247,7 @@ public class TestTaskExecution {
TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
TestProcessor.CONF_EMPTY);
// Setup the executor
- Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+ Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
// Signal the processor to go through
TestProcessor.awaitStart();
umbilical.signalThrowException();
@@ -280,7 +259,7 @@ public class TestTaskExecution {
} catch (ExecutionException e) {
Throwable cause = e.getCause();
assertTrue(cause instanceof IOException);
- assertTrue(cause.getMessage().contains(HEARTBEAT_EXCEPTION_STRING));
+ assertTrue(cause.getMessage().contains(TaskExecutionTestHelpers.HEARTBEAT_EXCEPTION_STRING));
}
TestProcessor.awaitCompletion();
assertTrue(TestProcessor.wasInterrupted());
@@ -307,7 +286,7 @@ public class TestTaskExecution {
TezTaskRunner taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
TestProcessor.CONF_EMPTY);
// Setup the executor
- Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable(taskRunner));
+ Future<Boolean> taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable1ForTest(taskRunner));
// Signal the processor to go through
TestProcessor.awaitStart();
umbilical.signalSendShouldDie();
@@ -329,38 +308,14 @@ public class TestTaskExecution {
}
}
- @Test(timeout = 5000)
- public void testGetTaskShouldDie() throws InterruptedException, ExecutionException {
- ListeningExecutorService executor = null;
- try {
- ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
- executor = MoreExecutors.listeningDecorator(rawExecutor);
- ApplicationId appId = ApplicationId.newInstance(10000, 1);
- ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
- ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
-
- TezTaskUmbilicalForTest umbilical = new TezTaskUmbilicalForTest();
- ContainerContext containerContext = new ContainerContext(containerId.toString());
-
- ContainerReporter containerReporter = new ContainerReporter(umbilical, containerContext, 100);
- ListenableFuture<ContainerTask> getTaskFuture = executor.submit(containerReporter);
-
- getTaskFuture.get();
- assertEquals(1, umbilical.getTaskInvocations);
-
- } finally {
- executor.shutdownNow();
- }
- }
-
// Potential new tests
// Different states - initialization failure, close failure
// getTask states
- private static class TaskRunnerCallable implements Callable<Boolean> {
+ private static class TaskRunnerCallable1ForTest implements Callable<Boolean> {
private final TezTaskRunner taskRunner;
- public TaskRunnerCallable(TezTaskRunner taskRunner) {
+ public TaskRunnerCallable1ForTest(TezTaskRunner taskRunner) {
this.taskRunner = taskRunner;
}
@@ -370,328 +325,9 @@ public class TestTaskExecution {
}
}
- // Uses static fields for signaling. Ensure only used by one test at a time.
- public static class TestProcessor extends AbstractLogicalIOProcessor {
-
- public static final byte[] CONF_EMPTY = new byte[] { 0 };
- public static final byte[] CONF_THROW_IO_EXCEPTION = new byte[] { 1 };
- public static final byte[] CONF_THROW_TEZ_EXCEPTION = new byte[] { 2 };
- public static final byte[] CONF_SIGNAL_FATAL_AND_THROW = new byte[] { 4 };
- public static final byte[] CONF_SIGNAL_FATAL_AND_LOOP = new byte[] { 8 };
- public static final byte[] CONF_SIGNAL_FATAL_AND_COMPLETE = new byte[] { 16 };
-
- private static final Logger LOG = LoggerFactory.getLogger(TestProcessor.class);
-
- private static final ReentrantLock processorLock = new ReentrantLock();
- private static final Condition processorCondition = processorLock.newCondition();
- private static final Condition completionCondition = processorLock.newCondition();
- private static final Condition runningCondition = processorLock.newCondition();
- private static boolean completed = false;
- private static boolean running = false;
- private static boolean signalled = false;
-
- public static boolean receivedInterrupt = false;
-
- private boolean throwIOException = false;
- private boolean throwTezException = false;
- private boolean signalFatalAndThrow = false;
- private boolean signalFatalAndLoop = false;
- private boolean signalFatalAndComplete = false;
-
- public TestProcessor(ProcessorContext context) {
- super(context);
- }
-
- @Override
- public void initialize() throws Exception {
- parseConf(getContext().getUserPayload().deepCopyAsArray());
- }
-
- @Override
- public void handleEvents(List<Event> processorEvents) {
-
- }
-
- @Override
- public void close() throws Exception {
-
- }
- private void parseConf(byte[] bytes) {
- byte b = bytes[0];
- throwIOException = (b & 1) > 1;
- throwTezException = (b & 2) > 1;
- signalFatalAndThrow = (b & 4) > 1;
- signalFatalAndLoop = (b & 8) > 1;
- signalFatalAndComplete = (b & 16) > 1;
- }
- public static void reset() {
- signalled = false;
- receivedInterrupt = false;
- completed = false;
- running = false;
- }
- public static void signal() {
- LOG.info("Signalled");
- processorLock.lock();
- try {
- signalled = true;
- processorCondition.signal();
- } finally {
- processorLock.unlock();
- }
- }
-
- public static void awaitStart() throws InterruptedException {
- LOG.info("Awaiting Process run");
- processorLock.lock();
- try {
- if (running) {
- return;
- }
- runningCondition.await();
- } finally {
- processorLock.unlock();
- }
- }
-
- public static void awaitCompletion() throws InterruptedException {
- LOG.info("Await completion");
- processorLock.lock();
- try {
- if (completed) {
- return;
- } else {
- completionCondition.await();
- }
- } finally {
- processorLock.unlock();
- }
- }
-
- public static boolean wasInterrupted() {
- processorLock.lock();
- try {
- return receivedInterrupt;
- } finally {
- processorLock.unlock();
- }
- }
-
- @Override
- public void run(Map<String, LogicalInput> inputs, Map<String, LogicalOutput> outputs) throws
- Exception {
- processorLock.lock();
- running = true;
- runningCondition.signal();
- try {
- try {
- LOG.info("Signal is: " + signalled);
- if (!signalled) {
- LOG.info("Waiting for processor signal");
- processorCondition.await();
- }
- if (Thread.currentThread().isInterrupted()) {
- throw new InterruptedException();
- }
- LOG.info("Received processor signal");
- if (throwIOException) {
- throw new IOException();
- } else if (throwTezException) {
- throw new TezException("TezException");
- } else if (signalFatalAndThrow) {
- IOException io = new IOException("FATALERROR");
- getContext().fatalError(io, "FATALERROR");
- throw io;
- } else if (signalFatalAndComplete) {
- IOException io = new IOException("FATALERROR");
- getContext().fatalError(io, "FATALERROR");
- return;
- } else if (signalFatalAndLoop) {
- IOException io = new IOException("FATALERROR");
- getContext().fatalError(io, "FATALERROR");
- LOG.info("Waiting for Processor signal again");
- processorCondition.await();
- LOG.info("Received second processor signal");
- }
- } catch (InterruptedException e) {
- receivedInterrupt = true;
- }
- } finally {
- completed = true;
- completionCondition.signal();
- processorLock.unlock();
- }
- }
- }
-
- private static class TezTaskUmbilicalForTest implements TezTaskUmbilicalProtocol {
-
- private static final Logger LOG = LoggerFactory.getLogger(TezTaskUmbilicalForTest.class);
-
- private final List<TezEvent> requestEvents = new LinkedList<TezEvent>();
-
- private final ReentrantLock umbilicalLock = new ReentrantLock();
- private final Condition eventCondition = umbilicalLock.newCondition();
- private boolean pendingEvent = false;
- private boolean eventEnacted = false;
-
- volatile int getTaskInvocations = 0;
-
- private boolean shouldThrowException = false;
- private boolean shouldSendDieSignal = false;
-
- public void signalThrowException() {
- umbilicalLock.lock();
- try {
- shouldThrowException = true;
- pendingEvent = true;
- } finally {
- umbilicalLock.unlock();
- }
- }
-
- public void signalSendShouldDie() {
- umbilicalLock.lock();
- try {
- shouldSendDieSignal = true;
- pendingEvent = true;
- } finally {
- umbilicalLock.unlock();
- }
- }
-
- public void awaitRegisteredEvent() throws InterruptedException {
- umbilicalLock.lock();
- try {
- if (eventEnacted) {
- return;
- }
- LOG.info("Awaiting event");
- eventCondition.await();
- } finally {
- umbilicalLock.unlock();
- }
- }
-
- public void resetTrackedEvents() {
- umbilicalLock.lock();
- try {
- requestEvents.clear();
- } finally {
- umbilicalLock.unlock();
- }
- }
-
- public void verifyNoCompletionEvents() {
- umbilicalLock.lock();
- try {
- for (TezEvent event : requestEvents) {
- if (event.getEvent() instanceof TaskAttemptFailedEvent) {
- fail("Found a TaskAttemptFailedEvent when not expected");
- }
- if (event.getEvent() instanceof TaskAttemptCompletedEvent) {
- fail("Found a TaskAttemptCompletedvent when not expected");
- }
- }
- } finally {
- umbilicalLock.unlock();
- }
- }
-
- public void verifyTaskFailedEvent(String diagnostics) {
- umbilicalLock.lock();
- try {
- for (TezEvent event : requestEvents) {
- if (event.getEvent() instanceof TaskAttemptFailedEvent) {
- TaskAttemptFailedEvent failedEvent = (TaskAttemptFailedEvent)event.getEvent();
- if(failedEvent.getDiagnostics().startsWith(diagnostics)){
- return ;
- } else {
- fail("No detailed diagnostics message in TaskAttemptFailedEvent");
- }
- }
- }
- fail("No TaskAttemptFailedEvents sent over umbilical");
- } finally {
- umbilicalLock.unlock();
- }
- }
-
- public void verifyTaskSuccessEvent() {
- umbilicalLock.lock();
- try {
- for (TezEvent event : requestEvents) {
- if (event.getEvent() instanceof TaskAttemptCompletedEvent) {
- return;
- }
- }
- fail("No TaskAttemptFailedEvents sent over umbilical");
- } finally {
- umbilicalLock.unlock();
- }
- }
-
- @Override
- public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
- return 0;
- }
-
- @Override
- public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
- int clientMethodsHash) throws IOException {
- return null;
- }
-
- @Override
- public ContainerTask getTask(ContainerContext containerContext) throws IOException {
- // Return shouldDie = true
- getTaskInvocations++;
- return new ContainerTask(null, true, null, null, false);
- }
-
- @Override
- public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
- return true;
- }
-
- @Override
- public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException,
- TezException {
- umbilicalLock.lock();
- if (request.getEvents() != null) {
- requestEvents.addAll(request.getEvents());
- }
- try {
- if (shouldThrowException) {
- LOG.info("TestUmbilical throwing Exception");
- throw new IOException(HEARTBEAT_EXCEPTION_STRING);
- }
- TezHeartbeatResponse response = new TezHeartbeatResponse();
- response.setLastRequestId(request.getRequestId());
- if (shouldSendDieSignal) {
- LOG.info("TestUmbilical returning shouldDie=true");
- response.setShouldDie();
- }
- return response;
- } finally {
- if (pendingEvent) {
- eventEnacted = true;
- LOG.info("Signalling Event");
- eventCondition.signal();
- }
- umbilicalLock.unlock();
- }
- }
- }
-
- private TaskReporter createTaskReporter(ApplicationId appId, TezTaskUmbilicalForTest umbilical) {
- TaskReporter taskReporter = new TaskReporter(umbilical, 100, 1000, 100, new AtomicLong(0),
- createContainerId(appId).toString());
- return taskReporter;
- }
private TezTaskRunner createTaskRunner(ApplicationId appId, TezTaskUmbilicalForTest umbilical,
TaskReporter taskReporter, ListeningExecutorService executor, byte[] processorConf)
@@ -722,9 +358,5 @@ public class TestTaskExecution {
return taskRunner;
}
- private ContainerId createContainerId(ApplicationId appId) {
- ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
- ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
- return containerId;
- }
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java
new file mode 100644
index 0000000..12d9d3f
--- /dev/null
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskExecution2.java
@@ -0,0 +1,638 @@
+/*
+ * 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.tez.runtime.task;
+
+import static org.apache.tez.runtime.task.TaskExecutionTestHelpers.createProcessorIOException;
+import static org.apache.tez.runtime.task.TaskExecutionTestHelpers.createProcessorTezException;
+import static org.apache.tez.runtime.task.TaskExecutionTestHelpers.createTaskReporter;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.ExecutionContext;
+import org.apache.tez.runtime.api.ObjectRegistry;
+import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.common.resources.ScalingAllocator;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
+import org.apache.tez.runtime.task.TaskExecutionTestHelpers.TestProcessor;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestTaskExecution2 {
+
+ private static final Logger LOG = LoggerFactory.getLogger(TestTaskExecution2.class);
+
+ private static final Configuration defaultConf = new Configuration();
+ private static final FileSystem localFs;
+ private static final Path workDir;
+
+ private static final ExecutorService taskExecutor = Executors.newFixedThreadPool(1);
+
+ static {
+ defaultConf.set("fs.defaultFS", "file:///");
+ defaultConf.set(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ALLOCATOR_CLASS,
+ ScalingAllocator.class.getName());
+ try {
+ localFs = FileSystem.getLocal(defaultConf);
+ Path wd = new Path(System.getProperty("test.build.data", "/tmp"),
+ TestTaskExecution.class.getSimpleName());
+ workDir = localFs.makeQualified(wd);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Before
+ public void reset() {
+ TestProcessor.reset();
+ }
+
+ @AfterClass
+ public static void shutdown() {
+ taskExecutor.shutdownNow();
+ }
+
+ @Test(timeout = 5000)
+ public void testSingleSuccessfulTask() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_EMPTY);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture = taskExecutor.submit(
+ new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.signal();
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.SUCCESS, null, false);
+ assertNull(taskReporter.currentCallable);
+ umbilical.verifyTaskSuccessEvent();
+ assertFalse(TestProcessor.wasAborted());
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testMultipleSuccessfulTasks() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_EMPTY);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture = taskExecutor.submit(
+ new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.signal();
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.SUCCESS, null, false);
+ assertNull(taskReporter.currentCallable);
+ umbilical.verifyTaskSuccessEvent();
+ assertFalse(TestProcessor.wasAborted());
+ umbilical.resetTrackedEvents();
+
+ taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_EMPTY);
+ // Setup the executor
+ taskRunnerFuture = taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.signal();
+ result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.SUCCESS, null, false);
+ assertNull(taskReporter.currentCallable);
+ umbilical.verifyTaskSuccessEvent();
+ assertFalse(TestProcessor.wasAborted());
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+ // test task failed due to exception in Processor
+ @Test(timeout = 5000)
+ public void testFailedTaskTezException() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_THROW_TEZ_EXCEPTION);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture =
+ taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.awaitStart();
+ TestProcessor.signal();
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.TASK_ERROR, createProcessorTezException(), false);
+
+ assertNull(taskReporter.currentCallable);
+ umbilical.verifyTaskFailedEvent(
+ "Failure while running task",
+ TezException.class.getName() + ": " + TezException.class.getSimpleName());
+ // Failure detected as a result of fall off from the run method. abort isn't required.
+ assertFalse(TestProcessor.wasAborted());
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+
+ // Test task failed due to Processor class not found
+ @Test(timeout = 5000)
+ public void testFailedTask2() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ "NotExitedProcessor", TestProcessor.CONF_EMPTY, false);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture =
+ taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.TASK_ERROR,
+ new TezUncheckedException("Unchecked exception"), false);
+
+ assertNull(taskReporter.currentCallable);
+ umbilical.verifyTaskFailedEvent("Failure while running task",
+ ":org.apache.tez.dag.api.TezUncheckedException: "
+ + "Unable to load class: NotExitedProcessor");
+ // Failure detected as a result of fall off from the run method. abort isn't required.
+ assertFalse(TestProcessor.wasAborted());
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+ // test task failed due to exception in Processor
+ @Test(timeout = 5000)
+ public void testFailedTaskIOException() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_THROW_IO_EXCEPTION);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture =
+ taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.awaitStart();
+ TestProcessor.signal();
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.TASK_ERROR, createProcessorIOException(), false);
+
+
+ assertNull(taskReporter.currentCallable);
+ umbilical.verifyTaskFailedEvent(
+ "Failure while running task",
+ IOException.class.getName() + ": " + IOException.class.getSimpleName());
+ // Failure detected as a result of fall off from the run method. abort isn't required.
+ assertFalse(TestProcessor.wasAborted());
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testHeartbeatException() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_EMPTY);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture =
+ taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.awaitStart();
+ umbilical.signalThrowException();
+ umbilical.awaitRegisteredEvent();
+ // Not signaling an actual start to verify task interruption
+
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.COMMUNICATION_FAILURE,
+ new IOException("IOException"),
+ TaskExecutionTestHelpers.HEARTBEAT_EXCEPTION_STRING, false);
+
+ TestProcessor.awaitCompletion();
+ assertTrue(TestProcessor.wasInterrupted());
+ assertNull(taskReporter.currentCallable);
+ // No completion events since umbilical communication already failed.
+ umbilical.verifyNoCompletionEvents();
+ assertTrue(TestProcessor.wasAborted());
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testHeartbeatShouldDie() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_EMPTY);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture =
+ taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.awaitStart();
+ umbilical.signalSendShouldDie();
+ umbilical.awaitRegisteredEvent();
+ // Not signaling an actual start to verify task interruption
+
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.CONTAINER_STOP_REQUESTED, null, true);
+
+
+ TestProcessor.awaitCompletion();
+ assertTrue(TestProcessor.wasInterrupted());
+ assertNull(taskReporter.currentCallable);
+ // TODO Is this statement correct ?
+ // No completion events since shouldDie was requested by the AM, which should have killed the
+ // task.
+ umbilical.verifyNoCompletionEvents();
+ assertTrue(TestProcessor.wasAborted());
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testSignalFatalErrorAndLoop() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_SIGNAL_FATAL_AND_LOOP);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture =
+ taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.awaitStart();
+ TestProcessor.signal();
+
+ TestProcessor.awaitLoop();
+ // The fatal error should have caused an interrupt.
+
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.TASK_ERROR, createProcessorIOException(), false);
+
+ TestProcessor.awaitCompletion();
+ assertTrue(TestProcessor.wasInterrupted());
+ assertNull(taskReporter.currentCallable);
+ umbilical.verifyTaskFailedEvent(
+ "Failure while running task",
+ IOException.class.getName() + ": " + IOException.class.getSimpleName());
+ // Signal fatal error should cause the processor to fail.
+ assertTrue(TestProcessor.wasAborted());
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testTaskKilled() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2 taskRunner = createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_EMPTY);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture =
+ taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.awaitStart();
+
+ taskRunner.killTask();
+
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.KILL_REQUESTED, null, false);
+
+ TestProcessor.awaitCompletion();
+ assertTrue(TestProcessor.wasInterrupted());
+ assertNull(taskReporter.currentCallable);
+ // Kill events are not sent over the umbilical at the moment.
+ umbilical.verifyNoCompletionEvents();
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testKilledAfterComplete() throws IOException, InterruptedException, TezException,
+ ExecutionException {
+
+ ListeningExecutorService executor = null;
+ try {
+ ExecutorService rawExecutor = Executors.newFixedThreadPool(1);
+ executor = MoreExecutors.listeningDecorator(rawExecutor);
+ ApplicationId appId = ApplicationId.newInstance(10000, 1);
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest
+ umbilical = new TaskExecutionTestHelpers.TezTaskUmbilicalForTest();
+ TaskReporter taskReporter = createTaskReporter(appId, umbilical);
+
+ TezTaskRunner2ForTest taskRunner =
+ createTaskRunnerForTest(appId, umbilical, taskReporter, executor,
+ TestProcessor.CONF_EMPTY);
+ // Setup the executor
+ Future<TaskRunner2Result> taskRunnerFuture =
+ taskExecutor.submit(new TaskRunnerCallable2ForTest(taskRunner));
+ // Signal the processor to go through
+ TestProcessor.awaitStart();
+ TestProcessor.signal();
+ TestProcessor.awaitCompletion();
+
+ taskRunner.awaitCallableCompletion();
+
+ taskRunner.killTask();
+ TaskRunner2Result result = taskRunnerFuture.get();
+ verifyTaskRunnerResult(result, EndReason.SUCCESS, null, false);
+
+ assertFalse(TestProcessor.wasInterrupted());
+ assertNull(taskReporter.currentCallable);
+ umbilical.verifyTaskSuccessEvent();
+ } finally {
+ executor.shutdownNow();
+ }
+ }
+
+
+ private void verifyTaskRunnerResult(TaskRunner2Result taskRunner2Result,
+ EndReason expectedEndReason, Throwable expectedThrowable,
+ boolean wasShutdownRequested) {
+ verifyTaskRunnerResult(taskRunner2Result, expectedEndReason, expectedThrowable, null,
+ wasShutdownRequested);
+ }
+
+ private void verifyTaskRunnerResult(TaskRunner2Result taskRunner2Result,
+ EndReason expectedEndReason, Throwable expectedThrowable,
+ String expectedExceptionMessage,
+ boolean wasShutdownRequested) {
+ assertEquals(expectedEndReason, taskRunner2Result.getEndReason());
+ if (expectedThrowable == null) {
+ assertNull(taskRunner2Result.getError());
+ } else {
+ assertNotNull(taskRunner2Result.getError());
+ Throwable cause = taskRunner2Result.getError();
+ LOG.info(cause.getClass().getName());
+ assertTrue(cause.getClass().isAssignableFrom(expectedThrowable.getClass()));
+
+ if (expectedExceptionMessage != null) {
+ assertTrue(cause.getMessage().contains(expectedExceptionMessage));
+ }
+
+ }
+ assertEquals(wasShutdownRequested, taskRunner2Result.isContainerShutdownRequested());
+ }
+
+
+ private static class TaskRunnerCallable2ForTest implements Callable<TaskRunner2Result> {
+ private final TezTaskRunner2 taskRunner;
+
+ public TaskRunnerCallable2ForTest(TezTaskRunner2 taskRunner) {
+ this.taskRunner = taskRunner;
+ }
+
+ @Override
+ public TaskRunner2Result call() throws Exception {
+ return taskRunner.run();
+ }
+ }
+
+ private TezTaskRunner2 createTaskRunner(ApplicationId appId,
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest umbilical,
+ TaskReporter taskReporter,
+ ListeningExecutorService executor, byte[] processorConf)
+ throws IOException {
+ return createTaskRunner(appId, umbilical, taskReporter, executor, TestProcessor.class.getName(),
+ processorConf, false);
+ }
+
+ private TezTaskRunner2ForTest createTaskRunnerForTest(ApplicationId appId,
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest umbilical,
+ TaskReporter taskReporter,
+ ListeningExecutorService executor,
+ byte[] processorConf)
+ throws IOException {
+ return (TezTaskRunner2ForTest) createTaskRunner(appId, umbilical, taskReporter, executor,
+ TestProcessor.class.getName(),
+ processorConf, true);
+ }
+
+ private TezTaskRunner2 createTaskRunner(ApplicationId appId,
+ TaskExecutionTestHelpers.TezTaskUmbilicalForTest umbilical,
+ TaskReporter taskReporter,
+ ListeningExecutorService executor, String processorClass,
+ byte[] processorConf, boolean testRunner) throws
+ IOException {
+ TezConfiguration tezConf = new TezConfiguration(defaultConf);
+ UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+ Path testDir = new Path(workDir, UUID.randomUUID().toString());
+ String[] localDirs = new String[]{testDir.toString()};
+
+ TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+ TezVertexID vertexId = TezVertexID.getInstance(dagId, 1);
+ TezTaskID taskId = TezTaskID.getInstance(vertexId, 1);
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 1);
+ ProcessorDescriptor processorDescriptor = ProcessorDescriptor.create(processorClass)
+ .setUserPayload(UserPayload.create(ByteBuffer.wrap(processorConf)));
+ TaskSpec taskSpec =
+ new TaskSpec(taskAttemptId, "dagName", "vertexName", -1, processorDescriptor,
+ new ArrayList<InputSpec>(), new ArrayList<OutputSpec>(), null);
+
+ TezTaskRunner2 taskRunner;
+ if (testRunner) {
+ taskRunner = new TezTaskRunner2ForTest(tezConf, ugi, localDirs, taskSpec, 1,
+ new HashMap<String, ByteBuffer>(), new HashMap<String, String>(),
+ HashMultimap.<String, String>create(), taskReporter,
+ executor, null, "", new ExecutionContextImpl("localhost"),
+ Runtime.getRuntime().maxMemory());
+ } else {
+ taskRunner = new TezTaskRunner2(tezConf, ugi, localDirs, taskSpec, 1,
+ new HashMap<String, ByteBuffer>(), new HashMap<String, String>(),
+ HashMultimap.<String, String>create(), taskReporter,
+ executor, null, "", new ExecutionContextImpl("localhost"),
+ Runtime.getRuntime().maxMemory());
+ }
+
+ return taskRunner;
+ }
+
+ public static class TezTaskRunner2ForTest extends TezTaskRunner2 {
+
+ private final ReentrantLock testLock = new ReentrantLock();
+ private final Condition callableCompletionCondition = testLock.newCondition();
+
+ private final AtomicBoolean isCallableComplete = new AtomicBoolean(false);
+
+ public TezTaskRunner2ForTest(Configuration tezConf, UserGroupInformation ugi,
+ String[] localDirs,
+ TaskSpec taskSpec, int appAttemptNumber,
+ Map<String, ByteBuffer> serviceConsumerMetadata,
+ Map<String, String> serviceProviderEnvMap,
+ Multimap<String, String> startedInputsMap,
+ TaskReporterInterface taskReporter,
+ ListeningExecutorService executor,
+ ObjectRegistry objectRegistry,
+ String pid,
+ ExecutionContext executionContext,
+ long memAvailable) throws IOException {
+ super(tezConf, ugi, localDirs, taskSpec, appAttemptNumber, serviceConsumerMetadata,
+ serviceProviderEnvMap, startedInputsMap, taskReporter, executor, objectRegistry, pid,
+ executionContext, memAvailable);
+ }
+
+
+ @Override
+ @VisibleForTesting
+ void processCallableResult(TaskRunner2Callable.TaskRunner2CallableResult executionResult) {
+ testLock.lock();
+ try {
+ super.processCallableResult(executionResult);
+ isCallableComplete.set(true);
+ callableCompletionCondition.signal();
+ } finally {
+ testLock.unlock();
+ }
+ }
+
+ void awaitCallableCompletion() throws InterruptedException {
+ testLock.lock();
+ try {
+ while (!isCallableComplete.get()) {
+ callableCompletionCondition.await();
+ }
+ } finally {
+ testLock.unlock();
+ }
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/eb82ca2c/tez-runtime-internals/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/resources/log4j.properties b/tez-runtime-internals/src/test/resources/log4j.properties
new file mode 100644
index 0000000..531b68b
--- /dev/null
+++ b/tez-runtime-internals/src/test/resources/log4j.properties
@@ -0,0 +1,19 @@
+# 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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n
[33/51] [abbrv] tez git commit: TEZ-2526. Fix version for
tez-history-parser. Contributed by Sergey Shelukhin.
Posted by ss...@apache.org.
TEZ-2526. Fix version for tez-history-parser. Contributed by Sergey Shelukhin.
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/b5459e24
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/b5459e24
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/b5459e24
Branch: refs/heads/TEZ-2003
Commit: b5459e2493971b98620121da4f146c85e743175e
Parents: c75bb37
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jun 2 12:47:58 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
tez-plugins/tez-history-parser/pom.xml | 2 +-
2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/b5459e24/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 55002fe..1e8abcf 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -31,5 +31,6 @@ ALL CHANGES:
TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
TEZ-2508. rebase 06/01
+ TEZ-2526. Fix version for tez-history-parser.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/b5459e24/tez-plugins/tez-history-parser/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml
index f12e0b4..5ab0090 100644
--- a/tez-plugins/tez-history-parser/pom.xml
+++ b/tez-plugins/tez-history-parser/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.tez</groupId>
<artifactId>tez-plugins</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.8.0-TEZ-2003-SNAPSHOT</version>
</parent>
<artifactId>tez-history-parser</artifactId>
[46/51] [abbrv] tez git commit: TEZ-2653. Change service contexts to
expose a user specified payload instead of the AM configuration. (sseth)
Posted by ss...@apache.org.
TEZ-2653. Change service contexts to expose a user specified payload
instead of the AM configuration. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/80e7053c
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/80e7053c
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/80e7053c
Branch: refs/heads/TEZ-2003
Commit: 80e7053c571318ba3d58094207ae087714e43715
Parents: 5afbef1
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Jul 28 14:56:56 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:58 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../tez/dag/api/NamedEntityDescriptor.java | 7 ++-
.../api/ContainerLauncherContext.java | 5 +-
.../api/TaskSchedulerContext.java | 5 +-
.../tez/dag/api/TaskCommunicatorContext.java | 4 +-
.../dag/app/ContainerLauncherContextImpl.java | 10 ++--
.../org/apache/tez/dag/app/DAGAppMaster.java | 20 +++++--
.../dag/app/TaskAttemptListenerImpTezDag.java | 19 ++++--
.../dag/app/TaskCommunicatorContextImpl.java | 12 ++--
.../tez/dag/app/TezTaskCommunicatorImpl.java | 12 +++-
.../dag/app/launcher/ContainerLauncherImpl.java | 8 ++-
.../app/launcher/ContainerLauncherRouter.java | 24 +++++---
.../app/launcher/LocalContainerLauncher.java | 10 +++-
.../dag/app/rm/LocalTaskSchedulerService.java | 10 +++-
.../dag/app/rm/TaskSchedulerContextImpl.java | 12 ++--
.../app/rm/TaskSchedulerContextImplWrapper.java | 6 +-
.../dag/app/rm/TaskSchedulerEventHandler.java | 61 ++++++++++++--------
.../dag/app/rm/YarnTaskSchedulerService.java | 21 +++++--
.../apache/tez/dag/app/MockDAGAppMaster.java | 15 ++++-
.../app/TestTaskAttemptListenerImplTezDag.java | 32 ++++++++--
.../app/TestTaskAttemptListenerImplTezDag2.java | 12 +++-
.../tez/dag/app/rm/TestContainerReuse.java | 19 +++---
.../tez/dag/app/rm/TestTaskScheduler.java | 3 +-
.../app/rm/TestTaskSchedulerEventHandler.java | 18 +++++-
.../dag/app/rm/TestTaskSchedulerHelpers.java | 26 +++++++--
.../TezTestServiceContainerLauncher.java | 15 +++--
.../rm/TezTestServiceTaskSchedulerService.java | 10 +++-
.../TezTestServiceTaskCommunicatorImpl.java | 2 +-
.../tez/tests/TestExternalTezServices.java | 13 ++++-
29 files changed, 294 insertions(+), 118 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index a201942..b88044b 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -38,5 +38,6 @@ ALL CHANGES:
TEZ-2005. Define basic interface for pluggable TaskScheduler.
TEZ-2651. Pluggable services should not extend AbstractService.
TEZ-2652. Cleanup the way services are specified for an AM and vertices.
+ TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
index bad0d10..723d43f 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
@@ -17,7 +17,7 @@ package org.apache.tez.dag.api;
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
-public class NamedEntityDescriptor<T extends EntityDescriptor<T>> extends EntityDescriptor<T> {
+public class NamedEntityDescriptor<T extends NamedEntityDescriptor<T>> extends EntityDescriptor<NamedEntityDescriptor<T>> {
private final String entityName;
@InterfaceAudience.Private
@@ -30,4 +30,9 @@ public class NamedEntityDescriptor<T extends EntityDescriptor<T>> extends Entity
public String getEntityName() {
return entityName;
}
+
+ public T setUserPayload(UserPayload userPayload) {
+ super.setUserPayload(userPayload);
+ return (T) this;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
index 836dc4a..5da38b8 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java
@@ -16,9 +16,9 @@ package org.apache.tez.serviceplugins.api;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.UserPayload;
@InterfaceAudience.Public
@InterfaceStability.Unstable
@@ -43,8 +43,7 @@ public interface ContainerLauncherContext {
// Lookup APIs
- // TODO TEZ-2003. To be replaced by getInitialPayload once the DAG API is changed.
- Configuration getInitialConfiguration();
+ UserPayload getInitialUserPayload();
int getNumNodes(String sourceName);
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
index b2c8799..6f37641 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
@@ -20,7 +20,6 @@ import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
@@ -30,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.UserPayload;
@InterfaceAudience.Public
@InterfaceStability.Unstable
@@ -84,8 +84,7 @@ public interface TaskSchedulerContext {
// Getters
- // TODO TEZ-2003. To be replaced by getInitialPayload
- public Configuration getInitialConfiguration();
+ public UserPayload getInitialUserPayload();
public String getAppTrackingUrl();
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index ab32ec1..a1e94a3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -18,7 +18,6 @@ import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Set;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -36,8 +35,7 @@ public interface TaskCommunicatorContext {
// TODO TEZ-2003 Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
- // TODO TEZ-2003 To be replaced by getInitialPayload
- Configuration getInitialConfiguration();
+ UserPayload getInitialUserPayload();
ApplicationAttemptId getApplicationAttemptId();
Credentials getCredentials();
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
index 997775a..92bbbdc 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ContainerLauncherContextImpl.java
@@ -14,10 +14,10 @@
package org.apache.tez.dag.app;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.tez.common.TezUtilsInternal;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.rm.container.AMContainerEvent;
@@ -33,10 +33,12 @@ public class ContainerLauncherContextImpl implements ContainerLauncherContext {
private final AppContext context;
private final TaskAttemptListener tal;
+ private final UserPayload initialUserPayload;
- public ContainerLauncherContextImpl(AppContext appContext, TaskAttemptListener tal) {
+ public ContainerLauncherContextImpl(AppContext appContext, TaskAttemptListener tal, UserPayload initialUserPayload) {
this.context = appContext;
this.tal = tal;
+ this.initialUserPayload = initialUserPayload;
}
@Override
@@ -76,8 +78,8 @@ public class ContainerLauncherContextImpl implements ContainerLauncherContext {
}
@Override
- public Configuration getInitialConfiguration() {
- return context.getAMConf();
+ public UserPayload getInitialUserPayload() {
+ return initialUserPayload;
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 8388cfb..4128841 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -62,8 +62,10 @@ import com.google.common.collect.HashBiMap;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.Options;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.SessionNotRunning;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
@@ -492,9 +494,12 @@ public class DAGAppMaster extends AbstractService {
jobTokenSecretManager.addTokenForJob(
appAttemptID.getApplicationId().toString(), sessionToken);
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(amConf);
+
//service to handle requests to TaskUmbilicalProtocol
taskAttemptListener = createTaskAttemptListener(context,
- taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors, isLocal);
+ taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors,
+ defaultPayload, isLocal);
addIfService(taskAttemptListener, true);
containerSignatureMatcher = createContainerSignatureMatcher();
@@ -540,9 +545,11 @@ public class DAGAppMaster extends AbstractService {
}
}
+
+
this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
- taskSchedulerDescriptors, isLocal);
+ taskSchedulerDescriptors, defaultPayload, isLocal);
addIfService(taskSchedulerEventHandler, true);
if (enableWebUIService()) {
@@ -560,7 +567,7 @@ public class DAGAppMaster extends AbstractService {
taskSchedulerEventHandler);
addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
- this.containerLauncherRouter = createContainerLauncherRouter(conf, containerLauncherDescriptors, isLocal);
+ this.containerLauncherRouter = createContainerLauncherRouter(defaultPayload, containerLauncherDescriptors, isLocal);
addIfService(containerLauncherRouter, true);
dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
@@ -1071,10 +1078,11 @@ public class DAGAppMaster extends AbstractService {
TaskHeartbeatHandler thh,
ContainerHeartbeatHandler chh,
List<NamedEntityDescriptor> entityDescriptors,
+ UserPayload defaultUserPayload,
boolean isLocal) {
TaskAttemptListener lis =
new TaskAttemptListenerImpTezDag(context, thh, chh,
- entityDescriptors, amConf, isLocal);
+ entityDescriptors, defaultUserPayload, isLocal);
return lis;
}
@@ -1095,11 +1103,11 @@ public class DAGAppMaster extends AbstractService {
return chh;
}
- protected ContainerLauncherRouter createContainerLauncherRouter(Configuration conf,
+ protected ContainerLauncherRouter createContainerLauncherRouter(UserPayload defaultPayload,
List<NamedEntityDescriptor> containerLauncherDescriptors,
boolean isLocal) throws
UnknownHostException {
- return new ContainerLauncherRouter(conf, context, taskAttemptListener, workingDirectory,
+ return new ContainerLauncherRouter(defaultPayload, context, taskAttemptListener, workingDirectory,
containerLauncherDescriptors, isLocal);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 1e34184..cc109a6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -29,9 +29,9 @@ import java.util.concurrent.ConcurrentMap;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import org.apache.commons.collections4.ListUtils;
-import org.apache.hadoop.conf.Configuration;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
@@ -103,7 +103,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
public TaskAttemptListenerImpTezDag(AppContext context,
TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
List<NamedEntityDescriptor> taskCommunicatorDescriptors,
- Configuration conf,
+ UserPayload defaultUserPayload,
boolean isPureLocalMode) {
super(TaskAttemptListenerImpTezDag.class.getName());
this.context = context;
@@ -112,17 +112,26 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
if (taskCommunicatorDescriptors == null || taskCommunicatorDescriptors.isEmpty()) {
if (isPureLocalMode) {
taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
- TezConstants.getTezUberServicePluginName(), null));
+ TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultUserPayload));
} else {
taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
- TezConstants.getTezYarnServicePluginName(), null));
+ TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultUserPayload));
}
}
this.taskCommunicators = new TaskCommunicator[taskCommunicatorDescriptors.size()];
this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorDescriptors.size()];
this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorDescriptors.size()];
for (int i = 0 ; i < taskCommunicatorDescriptors.size() ; i++) {
- taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, conf, i);
+ UserPayload userPayload;
+ if (taskCommunicatorDescriptors.get(i).getEntityName()
+ .equals(TezConstants.getTezYarnServicePluginName()) ||
+ taskCommunicatorDescriptors.get(i).getEntityName()
+ .equals(TezConstants.getTezUberServicePluginName())) {
+ userPayload = defaultUserPayload;
+ } else {
+ userPayload = taskCommunicatorDescriptors.get(i).getUserPayload();
+ }
+ taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, userPayload, i);
taskCommunicators[i] = createTaskCommunicator(taskCommunicatorDescriptors.get(i), i);
taskCommunicatorServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskCommunicators[i]);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index 035db93..cc315b7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -23,10 +23,10 @@ import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -49,17 +49,17 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
private final int taskCommunicatorIndex;
private final ReentrantReadWriteLock.ReadLock dagChangedReadLock;
private final ReentrantReadWriteLock.WriteLock dagChangedWriteLock;
- private final Configuration conf;
+ private final UserPayload userPayload;
private DAG dag;
public TaskCommunicatorContextImpl(AppContext appContext,
TaskAttemptListenerImpTezDag taskAttemptListener,
- Configuration conf,
+ UserPayload userPayload,
int taskCommunicatorIndex) {
this.context = appContext;
this.taskAttemptListener = taskAttemptListener;
- this.conf = conf;
+ this.userPayload = userPayload;
this.taskCommunicatorIndex = taskCommunicatorIndex;
ReentrantReadWriteLock dagChangedLock = new ReentrantReadWriteLock();
@@ -68,8 +68,8 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
}
@Override
- public Configuration getInitialConfiguration() {
- return conf;
+ public UserPayload getInitialUserPayload() {
+ return userPayload;
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 93b5b43..2a5c80e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -77,6 +77,7 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
protected final String tokenIdentifier;
protected final Token<JobTokenIdentifier> sessionToken;
+ protected final Configuration conf;
protected InetSocketAddress address;
protected volatile Server server;
@@ -119,6 +120,12 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
this.taskUmbilical = new TezTaskUmbilicalProtocolImpl();
this.tokenIdentifier = taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString();
this.sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
+ try {
+ conf = TezUtils.createConfFromUserPayload(getContext().getInitialUserPayload());
+ } catch (IOException e) {
+ throw new TezUncheckedException(
+ "Unable to parse user payload for " + TezTaskCommunicatorImpl.class.getSimpleName(), e);
+ }
}
@Override
@@ -132,7 +139,6 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
}
protected void startRpcServer() {
- Configuration conf = getContext().getInitialConfiguration();
try {
JobTokenSecretManager jobTokenSecretManager =
new JobTokenSecretManager();
@@ -171,6 +177,10 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
}
}
+ protected Configuration getConf() {
+ return this.conf;
+ }
+
private void refreshServiceAcls(Configuration configuration,
PolicyProvider policyProvider) {
this.server.refreshServiceAcl(configuration, policyProvider);
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
index cba5c80..07d269d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherImpl.java
@@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
@@ -224,7 +225,12 @@ public class ContainerLauncherImpl extends ContainerLauncher {
public ContainerLauncherImpl(ContainerLauncherContext containerLauncherContext) {
super(containerLauncherContext);
- this.conf = new Configuration(containerLauncherContext.getInitialConfiguration());
+ try {
+ this.conf = TezUtils.createConfFromUserPayload(containerLauncherContext.getInitialUserPayload());
+ } catch (IOException e) {
+ throw new TezUncheckedException(
+ "Failed to parse user payload for " + ContainerLauncherImpl.class.getSimpleName(), e);
+ }
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
0);
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 594e6d3..2d56bfe 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
@@ -63,7 +64,7 @@ public class ContainerLauncherRouter extends AbstractService
}
// Accepting conf to setup final parameters, if required.
- public ContainerLauncherRouter(Configuration conf, AppContext context,
+ public ContainerLauncherRouter(UserPayload defaultUserPayload, AppContext context,
TaskAttemptListener taskAttemptListener,
String workingDirectory,
List<NamedEntityDescriptor> containerLauncherDescriptors,
@@ -74,10 +75,10 @@ public class ContainerLauncherRouter extends AbstractService
if (containerLauncherDescriptors == null || containerLauncherDescriptors.isEmpty()) {
if (isPureLocalMode) {
containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
- TezConstants.getTezUberServicePluginName(), null));
+ TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultUserPayload));
} else {
containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
- TezConstants.getTezYarnServicePluginName(), null));
+ TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultUserPayload));
}
}
containerLauncherContexts = new ContainerLauncherContext[containerLauncherDescriptors.size()];
@@ -86,10 +87,20 @@ public class ContainerLauncherRouter extends AbstractService
for (int i = 0; i < containerLauncherDescriptors.size(); i++) {
- ContainerLauncherContext containerLauncherContext = new ContainerLauncherContextImpl(context, taskAttemptListener);
+ UserPayload userPayload;
+ if (containerLauncherDescriptors.get(i).getEntityName()
+ .equals(TezConstants.getTezYarnServicePluginName()) ||
+ containerLauncherDescriptors.get(i).getEntityName()
+ .equals(TezConstants.getTezUberServicePluginName())) {
+ userPayload = defaultUserPayload;
+ } else {
+ userPayload = containerLauncherDescriptors.get(i).getUserPayload();
+ }
+ ContainerLauncherContext containerLauncherContext =
+ new ContainerLauncherContextImpl(context, taskAttemptListener, userPayload);
containerLauncherContexts[i] = containerLauncherContext;
containerLaunchers[i] = createContainerLauncher(containerLauncherDescriptors.get(i), context,
- containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode, conf);
+ containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode);
containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
}
}
@@ -99,8 +110,7 @@ public class ContainerLauncherRouter extends AbstractService
ContainerLauncherContext containerLauncherContext,
TaskAttemptListener taskAttemptListener,
String workingDirectory,
- boolean isPureLocalMode,
- Configuration conf) throws
+ boolean isPureLocalMode) throws
UnknownHostException {
if (containerLauncherDescriptor.getEntityName().equals(
TezConstants.getTezYarnServicePluginName())) {
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index 3975111..1d3e6df 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -44,6 +44,7 @@ import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -129,7 +130,14 @@ public class LocalContainerLauncher extends ContainerLauncher {
System.getenv(Environment.NM_HOST.name());
executionContext = new ExecutionContextImpl(host);
- numExecutors = getContext().getInitialConfiguration().getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
+ Configuration conf;
+ try {
+ conf = TezUtils.createConfFromUserPayload(getContext().getInitialUserPayload());
+ } catch (IOException e) {
+ throw new TezUncheckedException(
+ "Failed to parse user payload for " + LocalContainerLauncher.class.getSimpleName(), e);
+ }
+ numExecutors = conf.getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS_DEFAULT);
Preconditions.checkState(numExecutors >=1, "Must have at least 1 executor");
ExecutorService rawExecutor = Executors.newFixedThreadPool(numExecutors,
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
index 476d00c..befde94 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
@@ -18,6 +18,7 @@
package org.apache.tez.dag.app.rm;
+import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.PriorityBlockingQueue;
@@ -27,6 +28,7 @@ import java.util.LinkedHashMap;
import com.google.common.primitives.Ints;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.serviceplugins.api.TaskScheduler;
import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
import org.slf4j.Logger;
@@ -65,7 +67,13 @@ public class LocalTaskSchedulerService extends TaskScheduler {
this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher();
this.customContainerAppId = taskSchedulerContext.getCustomClusterIdentifier();
- this.conf = taskSchedulerContext.getInitialConfiguration();
+ try {
+ this.conf = TezUtils.createConfFromUserPayload(taskSchedulerContext.getInitialUserPayload());
+ } catch (IOException e) {
+ throw new TezUncheckedException(
+ "Failed to deserialize payload for " + LocalTaskSchedulerService.class.getSimpleName(),
+ e);
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
index 890870e..7f1d5a3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
@@ -18,7 +18,6 @@ import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
@@ -28,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.common.ContainerSignatureMatcher;
import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
@@ -40,12 +40,12 @@ public class TaskSchedulerContextImpl implements TaskSchedulerContext {
private final long customClusterIdentifier;
private final String appHostName;
private final int clientPort;
- private final Configuration conf;
+ private final UserPayload initialUserPayload;
public TaskSchedulerContextImpl(TaskSchedulerEventHandler tseh, AppContext appContext,
int schedulerId, String trackingUrl, long customClusterIdentifier,
String appHostname, int clientPort,
- Configuration conf) {
+ UserPayload initialUserPayload) {
this.tseh = tseh;
this.appContext = appContext;
this.schedulerId = schedulerId;
@@ -53,7 +53,7 @@ public class TaskSchedulerContextImpl implements TaskSchedulerContext {
this.customClusterIdentifier = customClusterIdentifier;
this.appHostName = appHostname;
this.clientPort = clientPort;
- this.conf = conf;
+ this.initialUserPayload = initialUserPayload;
}
@@ -110,8 +110,8 @@ public class TaskSchedulerContextImpl implements TaskSchedulerContext {
}
@Override
- public Configuration getInitialConfiguration() {
- return conf;
+ public UserPayload getInitialUserPayload() {
+ return initialUserPayload;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
index e64ef43..9e4c8e0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
@@ -27,7 +27,6 @@ import java.util.concurrent.Future;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
@@ -37,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.common.ContainerSignatureMatcher;
import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
/**
@@ -132,8 +132,8 @@ class TaskSchedulerContextImplWrapper implements TaskSchedulerContext {
// does not use locks.
@Override
- public Configuration getInitialConfiguration() {
- return real.getInitialConfiguration();
+ public UserPayload getInitialUserPayload() {
+ return real.getInitialUserPayload();
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 67a088e..4c2e631 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -37,6 +37,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.serviceplugins.api.TaskScheduler;
import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
@@ -114,7 +115,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
private AtomicBoolean shouldUnregisterFlag =
new AtomicBoolean(false);
private final WebUIService webUI;
- private final String[] taskSchedulerClasses;
+ private final NamedEntityDescriptor[] taskSchedulerDescriptors;
protected final TaskScheduler[]taskSchedulers;
protected final ServicePluginLifecycleAbstractService []taskSchedulerServiceWrappers;
@@ -152,7 +153,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
public TaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
- List<NamedEntityDescriptor> schedulerDescriptors, boolean isPureLocalMode) {
+ List<NamedEntityDescriptor> schedulerDescriptors, UserPayload defaultPayload,
+ boolean isPureLocalMode) {
super(TaskSchedulerEventHandler.class.getName());
this.appContext = appContext;
this.eventHandler = eventHandler;
@@ -168,39 +170,50 @@ public class TaskSchedulerEventHandler extends AbstractService implements
// Override everything for pure local mode
if (isPureLocalMode) {
- this.taskSchedulerClasses = new String[] {TezConstants.getTezUberServicePluginName()};
+ this.taskSchedulerDescriptors = new NamedEntityDescriptor[]{
+ new NamedEntityDescriptor(TezConstants.getTezUberServicePluginName(), null)
+ .setUserPayload(defaultPayload)};
this.yarnTaskSchedulerIndex = -1;
} else {
if (schedulerDescriptors == null || schedulerDescriptors.isEmpty()) {
- this.taskSchedulerClasses = new String[] {TezConstants.getTezYarnServicePluginName()};
+ this.taskSchedulerDescriptors = new NamedEntityDescriptor[]{
+ new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(defaultPayload)};
this.yarnTaskSchedulerIndex = 0;
} else {
// Ensure the YarnScheduler will be setup and note it's index. This will be responsible for heartbeats and YARN registration.
int foundYarnTaskSchedulerIndex = -1;
- List<String> taskSchedulerClassList = new LinkedList<>();
+ List<NamedEntityDescriptor> schedulerDescriptorList = new LinkedList<>();
for (int i = 0 ; i < schedulerDescriptors.size() ; i++) {
if (schedulerDescriptors.get(i).getEntityName().equals(
TezConstants.getTezYarnServicePluginName())) {
- taskSchedulerClassList.add(schedulerDescriptors.get(i).getEntityName());
+ schedulerDescriptorList.add(
+ new NamedEntityDescriptor(schedulerDescriptors.get(i).getEntityName(), null)
+ .setUserPayload(
+ defaultPayload));
foundYarnTaskSchedulerIndex = i;
} else if (schedulerDescriptors.get(i).getEntityName().equals(
TezConstants.getTezUberServicePluginName())) {
- taskSchedulerClassList.add(schedulerDescriptors.get(i).getEntityName());
+ schedulerDescriptorList.add(
+ new NamedEntityDescriptor(schedulerDescriptors.get(i).getEntityName(), null)
+ .setUserPayload(
+ defaultPayload));
} else {
- taskSchedulerClassList.add(schedulerDescriptors.get(i).getClassName());
+ schedulerDescriptorList.add(schedulerDescriptors.get(i));
}
}
if (foundYarnTaskSchedulerIndex == -1) {
- taskSchedulerClassList.add(YarnTaskSchedulerService.class.getName());
- foundYarnTaskSchedulerIndex = taskSchedulerClassList.size() -1;
+ schedulerDescriptorList.add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null).setUserPayload(
+ defaultPayload));
+ foundYarnTaskSchedulerIndex = schedulerDescriptorList.size() -1;
}
- this.taskSchedulerClasses = taskSchedulerClassList.toArray(new String[taskSchedulerClassList.size()]);
+ this.taskSchedulerDescriptors = schedulerDescriptorList.toArray(new NamedEntityDescriptor[schedulerDescriptorList.size()]);
this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
}
}
- taskSchedulers = new TaskScheduler[this.taskSchedulerClasses.length];
- taskSchedulerServiceWrappers = new ServicePluginLifecycleAbstractService[this.taskSchedulerClasses.length];
+ taskSchedulers = new TaskScheduler[this.taskSchedulerDescriptors.length];
+ taskSchedulerServiceWrappers = new ServicePluginLifecycleAbstractService[this.taskSchedulerDescriptors.length];
}
public Map<ApplicationAccessType, String> getApplicationAcls() {
@@ -417,23 +430,24 @@ public class TaskSchedulerEventHandler extends AbstractService implements
private TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
AppContext appContext,
- String schedulerClassName,
+ NamedEntityDescriptor taskSchedulerDescriptor,
long customAppIdIdentifier,
int schedulerId) {
TaskSchedulerContext rawContext =
new TaskSchedulerContextImpl(this, appContext, schedulerId, trackingUrl,
- customAppIdIdentifier, host, port, getConfig());
+ customAppIdIdentifier, host, port, taskSchedulerDescriptor.getUserPayload());
TaskSchedulerContext wrappedContext = new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
- if (schedulerClassName.equals(TezConstants.getTezYarnServicePluginName())) {
+ String schedulerName = taskSchedulerDescriptor.getEntityName();
+ if (schedulerName.equals(TezConstants.getTezYarnServicePluginName())) {
LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
return new YarnTaskSchedulerService(wrappedContext);
- } else if (schedulerClassName.equals(TezConstants.getTezUberServicePluginName())) {
+ } else if (schedulerName.equals(TezConstants.getTezUberServicePluginName())) {
LOG.info("Creating TaskScheduler: Local TaskScheduler");
return new LocalTaskSchedulerService(wrappedContext);
} else {
- LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
+ LOG.info("Creating custom TaskScheduler {}:{}", taskSchedulerDescriptor.getEntityName(), taskSchedulerDescriptor.getClassName());
Class<? extends TaskScheduler> taskSchedulerClazz =
- (Class<? extends TaskScheduler>) ReflectionUtils.getClazz(schedulerClassName);
+ (Class<? extends TaskScheduler>) ReflectionUtils.getClazz(taskSchedulerDescriptor.getClassName());
try {
Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
.getConstructor(TaskSchedulerContext.class);
@@ -453,21 +467,20 @@ public class TaskSchedulerEventHandler extends AbstractService implements
@VisibleForTesting
protected void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
- // TODO Add error checking for components being used in the Vertex when running in pure local mode.
// Iterate over the list and create all the taskSchedulers
int j = 0;
- for (int i = 0; i < taskSchedulerClasses.length; i++) {
+ for (int i = 0; i < taskSchedulerDescriptors.length; i++) {
long customAppIdIdentifier;
- if (isPureLocalMode || taskSchedulerClasses[i].equals(
+ if (isPureLocalMode || taskSchedulerDescriptors[i].equals(
TezConstants.getTezYarnServicePluginName())) { // Use the app identifier from the appId.
customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp();
} else {
customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT);
}
- LOG.info("ClusterIdentifier for TaskScheduler [" + i + ":" + taskSchedulerClasses[i] + "]=" +
+ LOG.info("ClusterIdentifier for TaskScheduler [" + i + ":" + taskSchedulerDescriptors[i].getEntityName() + "]=" +
customAppIdIdentifier);
taskSchedulers[i] = createTaskScheduler(host, port,
- trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier, i);
+ trackingUrl, appContext, taskSchedulerDescriptors[i], customAppIdIdentifier, i);
taskSchedulerServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskSchedulers[i]);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
index 1e76dc9..940c5b0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
@@ -30,13 +30,11 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
import java.util.concurrent.PriorityBlockingQueue;
-import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.serviceplugins.api.TaskScheduler;
import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AMState;
@@ -70,7 +68,6 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
/* TODO not yet updating cluster nodes on every allocate response
* from RMContainerRequestor
@@ -218,7 +215,13 @@ public class YarnTaskSchedulerService extends TaskScheduler
this.appHostName = taskSchedulerContext.getAppHostName();
this.appHostPort = taskSchedulerContext.getAppClientPort();
this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
- this.conf = taskSchedulerContext.getInitialConfiguration();
+ try {
+ this.conf = TezUtils.createConfFromUserPayload(taskSchedulerContext.getInitialUserPayload());
+ } catch (IOException e) {
+ throw new TezUncheckedException(
+ "Failed to deserialize payload for " + YarnTaskSchedulerService.class.getSimpleName(),
+ e);
+ }
}
@Private
@@ -231,7 +234,13 @@ public class YarnTaskSchedulerService extends TaskScheduler
this.appHostName = taskSchedulerContext.getAppHostName();
this.appHostPort = taskSchedulerContext.getAppClientPort();
this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
- this.conf = taskSchedulerContext.getInitialConfiguration();
+ try {
+ this.conf = TezUtils.createConfFromUserPayload(taskSchedulerContext.getInitialUserPayload());
+ } catch (IOException e) {
+ throw new TezUncheckedException(
+ "Failed to deserialize payload for " + YarnTaskSchedulerService.class.getSimpleName(),
+ e);
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 17feeaa..0723dbc 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -18,6 +18,7 @@
package org.apache.tez.dag.app;
+import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.ThreadMXBean;
import java.net.UnknownHostException;
@@ -34,7 +35,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -488,7 +491,15 @@ public class MockDAGAppMaster extends DAGAppMaster {
super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
isSession, workingDirectory, localDirs, logDirs, new TezApiVersionInfo().getVersion(), 1,
credentials, jobUserName, null);
- containerLauncherContext = new ContainerLauncherContextImpl(getContext(), getTaskAttemptListener());
+ Configuration conf = new Configuration(false);
+ UserPayload userPayload;
+ try {
+ userPayload = TezUtils.createUserPayloadFromConf(conf);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
+ containerLauncherContext =
+ new ContainerLauncherContextImpl(getContext(), getTaskAttemptListener(), userPayload);
containerLauncher = new MockContainerLauncher(launcherGoFlag, containerLauncherContext);
shutdownHandler = new MockDAGAppMasterShutdownHandler();
this.initFailFlag = initFailFlag;
@@ -500,7 +511,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
// use mock container launcher for tests
@Override
- protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
+ protected ContainerLauncherRouter createContainerLauncherRouter(final UserPayload defaultUserPayload,
List<NamedEntityDescriptor> containerLauncherDescirptors,
boolean isLocal)
throws UnknownHostException {
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index e45b0a2..1cb69a8 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -47,11 +47,14 @@ import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ContainerContext;
import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.common.security.TokenCache;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TaskHeartbeatRequest;
@@ -135,8 +138,15 @@ public class TestTaskAttemptListenerImplTezDag {
doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
doReturn(container).when(amContainer).getContainer();
+ Configuration conf = new TezConfiguration();
+ UserPayload defaultPayload;
+ try {
+ defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, defaultPayload, false);
TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
@@ -299,8 +309,14 @@ public class TestTaskAttemptListenerImplTezDag {
new JobTokenSecretManager());
sessionToken.setService(identifier.getJobId());
TokenCache.setSessionToken(sessionToken, credentials);
+ UserPayload userPayload = null;
+ try {
+ userPayload = TezUtils.createUserPayloadFromConf(conf);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, conf, false);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, userPayload, false);
// no exception happen, should started properly
taskAttemptListener.init(conf);
taskAttemptListener.start();
@@ -319,8 +335,14 @@ public class TestTaskAttemptListenerImplTezDag {
TokenCache.setSessionToken(sessionToken, credentials);
conf.set(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE, port + "-" + port);
+ UserPayload userPayload = null;
+ try {
+ userPayload = TezUtils.createUserPayloadFromConf(conf);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, conf, false);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, userPayload, false);
taskAttemptListener.init(conf);
taskAttemptListener.start();
int resultedPort = taskAttemptListener.getTaskCommunicator(0).getAddress().getPort();
@@ -377,9 +399,9 @@ public class TestTaskAttemptListenerImplTezDag {
TaskHeartbeatHandler thh,
ContainerHeartbeatHandler chh,
List<NamedEntityDescriptor> taskCommDescriptors,
- Configuration conf,
+ UserPayload userPayload,
boolean isPureLocalMode) {
- super(context, thh, chh, taskCommDescriptors, conf,
+ super(context, thh, chh, taskCommDescriptors, userPayload,
isPureLocalMode);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
index 8d776fb..1c82bd8 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
@@ -22,9 +22,11 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
+import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -34,6 +36,10 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
@@ -51,7 +57,7 @@ import org.mockito.ArgumentCaptor;
public class TestTaskAttemptListenerImplTezDag2 {
@Test(timeout = 5000)
- public void testTaskAttemptFailedKilled() {
+ public void testTaskAttemptFailedKilled() throws IOException {
ApplicationId appId = ApplicationId.newInstance(1000, 1);
ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
Credentials credentials = new Credentials();
@@ -73,9 +79,11 @@ public class TestTaskAttemptListenerImplTezDag2 {
doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
doReturn(container).when(amContainer).getContainer();
+ Configuration conf = new TezConfiguration();
+ UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
TaskAttemptListenerImpTezDag taskAttemptListener =
new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
- mock(ContainerHeartbeatHandler.class), null, null, false);
+ mock(ContainerHeartbeatHandler.class), null, userPayload, false);
TaskSpec taskSpec1 = mock(TaskSpec.class);
TezTaskAttemptID taskAttemptId1 = mock(TezTaskAttemptID.class);
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 88f6066..8e8224a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -37,6 +37,7 @@ import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.tez.common.TezUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -144,7 +145,7 @@ public class TestContainerReuse {
TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
new TaskSchedulerEventHandlerForTest(
appContext, eventHandler, rmClient,
- new AlwaysMatchesContainerMatcher());
+ new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(conf));
TaskSchedulerEventHandler taskSchedulerEventHandler =
spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(conf);
@@ -279,7 +280,7 @@ public class TestContainerReuse {
TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient,
- new AlwaysMatchesContainerMatcher());
+ new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(conf));
TaskSchedulerEventHandler taskSchedulerEventHandler =
spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(conf);
@@ -378,7 +379,7 @@ public class TestContainerReuse {
doReturn(dagID).when(appContext).getCurrentDAGID();
doReturn(mock(ClusterInfo.class)).when(appContext).getClusterInfo();
- TaskSchedulerEventHandler taskSchedulerEventHandlerReal = new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new AlwaysMatchesContainerMatcher());
+ TaskSchedulerEventHandler taskSchedulerEventHandlerReal = new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
@@ -514,7 +515,7 @@ public class TestContainerReuse {
//Use ContainerContextMatcher here. Otherwise it would not match the JVM options
TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
- new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new ContainerContextMatcher());
+ new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new ContainerContextMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
@@ -709,7 +710,7 @@ public class TestContainerReuse {
TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
new TaskSchedulerEventHandlerForTest(
appContext, eventHandler, rmClient,
- new AlwaysMatchesContainerMatcher());
+ new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
TaskSchedulerEventHandler taskSchedulerEventHandler =
spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(tezConf);
@@ -833,7 +834,7 @@ public class TestContainerReuse {
TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient,
- new AlwaysMatchesContainerMatcher());
+ new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
TaskSchedulerEventHandler taskSchedulerEventHandler =
spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(tezConf);
@@ -947,7 +948,7 @@ public class TestContainerReuse {
doAnswer(dagIDAnswer).when(appContext).getCurrentDAGID();
doReturn(mock(ClusterInfo.class)).when(appContext).getClusterInfo();
- TaskSchedulerEventHandler taskSchedulerEventHandlerReal = new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new AlwaysMatchesContainerMatcher());
+ TaskSchedulerEventHandler taskSchedulerEventHandlerReal = new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
@@ -1105,7 +1106,7 @@ public class TestContainerReuse {
TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient,
- new ContainerContextMatcher());
+ new ContainerContextMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
@@ -1259,7 +1260,7 @@ public class TestContainerReuse {
TaskSchedulerEventHandler taskSchedulerEventHandlerReal =
new TaskSchedulerEventHandlerForTest(appContext, eventHandler, rmClient,
- new AlwaysMatchesContainerMatcher());
+ new AlwaysMatchesContainerMatcher(), TezUtils.createUserPayloadFromConf(tezConf));
TaskSchedulerEventHandler taskSchedulerEventHandler = spy(taskSchedulerEventHandlerReal);
taskSchedulerEventHandler.init(tezConf);
taskSchedulerEventHandler.start();
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
index 123a4d7..6af9815 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
@@ -145,7 +145,8 @@ public class TestTaskScheduler {
scheduler.initialize();
drainableAppCallback.drain();
- verify(mockRMClient).init(conf);
+ // Verifying the validity of the configuration via the interval only instead of making sure
+ // it's the same instance.
verify(mockRMClient).setHeartbeatInterval(interval);
RegisterApplicationMasterResponse mockRegResponse =
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index f191175..f8aa1e2 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -49,9 +49,12 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.client.DAGClientServer;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ContainerContext;
@@ -92,8 +95,10 @@ public class TestTaskSchedulerEventHandler {
public MockTaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
- ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
- super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI, new LinkedList<NamedEntityDescriptor>(), false);
+ ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
+ UserPayload defaultPayload) {
+ super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI,
+ new LinkedList<NamedEntityDescriptor>(), defaultPayload, false);
}
@Override
@@ -134,8 +139,15 @@ public class TestTaskSchedulerEventHandler {
mockWebUIService = mock(WebUIService.class);
when(mockAppContext.getAllContainers()).thenReturn(mockAMContainerMap);
when(mockClientService.getBindAddress()).thenReturn(new InetSocketAddress(10000));
+ Configuration conf = new Configuration(false);
+ UserPayload userPayload;
+ try {
+ userPayload = TezUtils.createUserPayloadFromConf(conf);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
schedulerHandler = new MockTaskSchedulerEventHandler(
- mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService);
+ mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService, userPayload);
}
@Test(timeout = 5000)
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index 60d37e9..59ab00a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -24,6 +24,7 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
+import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.LinkedList;
@@ -59,7 +60,10 @@ import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest;
@@ -125,22 +129,26 @@ class TestTaskSchedulerHelpers {
private TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync;
private ContainerSignatureMatcher containerSignatureMatcher;
+ private UserPayload defaultPayload;
@SuppressWarnings("rawtypes")
public TaskSchedulerEventHandlerForTest(AppContext appContext,
EventHandler eventHandler,
TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
- ContainerSignatureMatcher containerSignatureMatcher) {
- super(appContext, null, eventHandler, containerSignatureMatcher, null, new LinkedList<NamedEntityDescriptor>(), false);
+ ContainerSignatureMatcher containerSignatureMatcher,
+ UserPayload defaultPayload) {
+ super(appContext, null, eventHandler, containerSignatureMatcher, null,
+ new LinkedList<NamedEntityDescriptor>(), defaultPayload, false);
this.amrmClientAsync = amrmClientAsync;
this.containerSignatureMatcher = containerSignatureMatcher;
+ this.defaultPayload = defaultPayload;
}
@Override
public void instantiateScheduelrs(String host, int port, String trackingUrl, AppContext appContext) {
TaskSchedulerContext taskSchedulerContext =
new TaskSchedulerContextImpl(this, appContext, 0, trackingUrl, 1000, host, port,
- getConfig());
+ defaultPayload);
TaskSchedulerContextImplWrapper wrapper =
new TaskSchedulerContextImplWrapper(taskSchedulerContext,
new CountingExecutorService(appCallbackExecutor));
@@ -287,8 +295,8 @@ class TestTaskSchedulerHelpers {
// Not incrementing invocations for methods which to not obtain locks,
// and do not go via the executor service.
@Override
- public Configuration getInitialConfiguration() {
- return real.getInitialConfiguration();
+ public UserPayload getInitialUserPayload() {
+ return real.getInitialUserPayload();
}
@Override
@@ -523,7 +531,13 @@ class TestTaskSchedulerHelpers {
when(mockContext.getAppTrackingUrl()).thenReturn(appUrl);
when(mockContext.getAMState()).thenReturn(TaskSchedulerContext.AMState.RUNNING_APP);
- when(mockContext.getInitialConfiguration()).thenReturn(conf);
+ UserPayload userPayload;
+ try {
+ userPayload = TezUtils.createUserPayloadFromConf(conf);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
+ when(mockContext.getInitialUserPayload()).thenReturn(userPayload);
when(mockContext.isSession()).thenReturn(isSession);
if (containerSignatureMatcher != null) {
when(mockContext.getContainerSignatureMatcher())
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index 0002b42..f31a07b 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -19,10 +19,12 @@ import java.net.InetSocketAddress;
import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.serviceplugins.api.ContainerLaunchRequest;
import org.apache.tez.serviceplugins.api.ContainerLauncher;
import org.apache.tez.serviceplugins.api.ContainerLauncherContext;
@@ -49,17 +51,22 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
private final int servicePort;
private final TezTestServiceCommunicator communicator;
private final ApplicationAttemptId appAttemptId;
- // private final TaskAttemptListener tal;
+ private final Configuration conf;
// Configuration passed in here to set up final parameters
public TezTestServiceContainerLauncher(ContainerLauncherContext containerLauncherContext) {
super(containerLauncherContext);
- int numThreads = getContext().getInitialConfiguration().getInt(
+ try {
+ conf = TezUtils.createConfFromUserPayload(getContext().getInitialUserPayload());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ int numThreads = conf.getInt(
TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS,
TezTestServiceConfConstants.TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT);
- this.servicePort = getContext().getInitialConfiguration().getInt(
+ this.servicePort = conf.getInt(
TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT, -1);
Preconditions.checkArgument(servicePort > 0,
TezTestServiceConfConstants.TEZ_TEST_SERVICE_RPC_PORT + " must be set");
@@ -70,7 +77,7 @@ public class TezTestServiceContainerLauncher extends ContainerLauncher {
@Override
public void start() {
- communicator.init(getContext().getInitialConfiguration());
+ communicator.init(conf);
communicator.start();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
index 7d209bc..0d87995 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/rm/TezTestServiceTaskSchedulerService.java
@@ -14,6 +14,7 @@
package org.apache.tez.dag.app.rm;
+import java.io.IOException;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
@@ -32,6 +33,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.service.TezTestServiceConfConstants;
import org.apache.tez.serviceplugins.api.TaskScheduler;
@@ -74,7 +77,12 @@ public class TezTestServiceTaskSchedulerService extends TaskScheduler {
this.containerFactory = new ContainerFactory(taskSchedulerContext.getApplicationAttemptId(),
taskSchedulerContext.getCustomClusterIdentifier());
- Configuration conf = taskSchedulerContext.getInitialConfiguration();
+ Configuration conf = null;
+ try {
+ conf = TezUtils.createConfFromUserPayload(taskSchedulerContext.getInitialUserPayload());
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
this.memoryPerInstance = conf
.getInt(TezTestServiceConfConstants.TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB, -1);
Preconditions.checkArgument(memoryPerInstance > 0,
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index 078ea79..ef8f9e4 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -76,7 +76,7 @@ public class TezTestServiceTaskCommunicatorImpl extends TezTaskCommunicatorImpl
@Override
public void initialize() throws Exception {
super.initialize();
- this.communicator.init(getContext().getInitialConfiguration());
+ this.communicator.init(getConf());
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/80e7053c/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
index 07dd363..2c52ae3 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -24,10 +24,12 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.tez.client.TezClient;
+import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.DAG;
import org.apache.tez.dag.api.ProcessorDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.Vertex;
import org.apache.tez.dag.api.Vertex.VertexExecutionContext;
import org.apache.tez.dag.api.client.DAGClient;
@@ -136,17 +138,22 @@ public class TestExternalTezServices {
confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
confForJobs.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, false);
+ UserPayload userPayload = TezUtils.createUserPayloadFromConf(confForJobs);
+
TaskSchedulerDescriptor[] taskSchedulerDescriptors = new TaskSchedulerDescriptor[]{
TaskSchedulerDescriptor
- .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskSchedulerService.class.getName())};
+ .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskSchedulerService.class.getName())
+ .setUserPayload(userPayload)};
ContainerLauncherDescriptor[] containerLauncherDescriptors = new ContainerLauncherDescriptor[]{
ContainerLauncherDescriptor
- .create(EXT_PUSH_ENTITY_NAME, TezTestServiceNoOpContainerLauncher.class.getName())};
+ .create(EXT_PUSH_ENTITY_NAME, TezTestServiceNoOpContainerLauncher.class.getName())
+ .setUserPayload(userPayload)};
TaskCommunicatorDescriptor[] taskCommunicatorDescriptors = new TaskCommunicatorDescriptor[]{
TaskCommunicatorDescriptor
- .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskCommunicatorImpl.class.getName())};
+ .create(EXT_PUSH_ENTITY_NAME, TezTestServiceTaskCommunicatorImpl.class.getName())
+ .setUserPayload(userPayload)};
ServicePluginsDescriptor servicePluginsDescriptor = ServicePluginsDescriptor.create(true, true,
taskSchedulerDescriptors, containerLauncherDescriptors, taskCommunicatorDescriptors);
[18/51] [abbrv] tez git commit: TEZ-2284. Separate TaskReporter into
an interface. (sseth)
Posted by ss...@apache.org.
TEZ-2284. Separate TaskReporter into an interface. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/08a196a1
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/08a196a1
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/08a196a1
Branch: refs/heads/TEZ-2003
Commit: 08a196a17f409024f410bb5c108bb1ead0099fb0
Parents: 1f75f32
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:21:35 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../internals/api/TaskReporterInterface.java | 46 ++++++++++++++++++++
.../apache/tez/runtime/task/TaskReporter.java | 12 ++++-
.../org/apache/tez/runtime/task/TezChild.java | 3 +-
.../apache/tez/runtime/task/TezTaskRunner.java | 5 ++-
5 files changed, 62 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/08a196a1/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 6a4399c..e2c428d 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -13,5 +13,6 @@ ALL CHANGES:
TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
TEZ-2241. Miscellaneous fixes after last reabse.
TEZ-2283. Fixes after rebase 04/07.
+ TEZ-2284. Separate TaskReporter into an interface.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/08a196a1/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
new file mode 100644
index 0000000..47a61ab
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TaskReporterInterface.java
@@ -0,0 +1,46 @@
+/*
+ * 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.tez.runtime.internals.api;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.task.ErrorReporter;
+
+public interface TaskReporterInterface {
+
+ // TODO TEZ-2003 Consolidate private API usage if making this public
+
+ void registerTask(RuntimeTask task, ErrorReporter errorReporter);
+
+ void unregisterTask(TezTaskAttemptID taskAttemptId);
+
+ boolean taskSucceeded(TezTaskAttemptID taskAttemptId) throws IOException, TezException;
+
+ boolean taskFailed(TezTaskAttemptID taskAttemptId, Throwable cause, String diagnostics, EventMetaData srcMeta) throws IOException,
+ TezException;
+
+ void addEvents(TezTaskAttemptID taskAttemptId, Collection<TezEvent> events);
+
+ boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
+
+ void shutdown();
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/08a196a1/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
index d9a7786..3579e3f 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
@@ -48,6 +48,7 @@ import org.apache.tez.runtime.api.impl.TezEvent;
import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -66,7 +67,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
* retrieve events specific to this task.
*
*/
-public class TaskReporter {
+public class TaskReporter implements TaskReporterInterface {
private static final Logger LOG = LoggerFactory.getLogger(TaskReporter.class);
@@ -98,6 +99,7 @@ public class TaskReporter {
/**
* Register a task to be tracked. Heartbeats will be sent out for this task to fetch events, etc.
*/
+ @Override
public synchronized void registerTask(RuntimeTask task,
ErrorReporter errorReporter) {
currentCallable = new HeartbeatCallable(task, umbilical, pollInterval, sendCounterInterval,
@@ -110,12 +112,14 @@ public class TaskReporter {
* This method should always be invoked before setting up heartbeats for another task running in
* the same container.
*/
+ @Override
public synchronized void unregisterTask(TezTaskAttemptID taskAttemptID) {
currentCallable.markComplete();
currentCallable = null;
// KKK Make sure the callable completes before proceeding
}
-
+
+ @Override
public void shutdown() {
heartbeatExecutor.shutdownNow();
}
@@ -413,19 +417,23 @@ public class TaskReporter {
}
}
+ @Override
public synchronized boolean taskSucceeded(TezTaskAttemptID taskAttemptID) throws IOException, TezException {
return currentCallable.taskSucceeded(taskAttemptID);
}
+ @Override
public synchronized boolean taskFailed(TezTaskAttemptID taskAttemptID, Throwable t, String diagnostics,
EventMetaData srcMeta) throws IOException, TezException {
return currentCallable.taskFailed(taskAttemptID, t, diagnostics, srcMeta);
}
+ @Override
public synchronized void addEvents(TezTaskAttemptID taskAttemptID, Collection<TezEvent> events) {
currentCallable.addEvents(taskAttemptID, events);
}
+ @Override
public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException {
return umbilical.canCommit(taskAttemptID);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/08a196a1/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index 32da8fb..4c8bebc 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -69,6 +69,7 @@ import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.api.impl.TezUmbilical;
import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -114,7 +115,7 @@ public class TezChild {
private final boolean ownUmbilical;
private final TezTaskUmbilicalProtocol umbilical;
- private TaskReporter taskReporter;
+ private TaskReporterInterface taskReporter;
private int taskCount = 0;
private TezVertexID lastVertexID;
http://git-wip-us.apache.org/repos/asf/tez/blob/08a196a1/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index f54814b..33a7f4a 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -41,6 +41,7 @@ import org.apache.tez.runtime.api.impl.EventMetaData;
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.api.impl.TezEvent;
import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.internals.api.TaskReporterInterface;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -56,7 +57,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
private final LogicalIOProcessorRuntimeTask task;
private final UserGroupInformation ugi;
- private final TaskReporter taskReporter;
+ private final TaskReporterInterface taskReporter;
private final ListeningExecutorService executor;
private volatile ListenableFuture<Void> taskFuture;
private volatile Thread waitingThread;
@@ -70,7 +71,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
public TezTaskRunner(Configuration tezConf, UserGroupInformation ugi, String[] localDirs,
TaskSpec taskSpec, int appAttemptNumber,
Map<String, ByteBuffer> serviceConsumerMetadata, Map<String, String> serviceProviderEnvMap,
- Multimap<String, String> startedInputsMap, TaskReporter taskReporter,
+ Multimap<String, String> startedInputsMap, TaskReporterInterface taskReporter,
ListeningExecutorService executor, ObjectRegistry objectRegistry, String pid,
ExecutionContext executionContext, long memAvailable)
throws IOException {
[32/51] [abbrv] tez git commit: TEZ-2508. rebase 06/01. (sseth)
Posted by ss...@apache.org.
TEZ-2508. rebase 06/01. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/c75bb377
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/c75bb377
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/c75bb377
Branch: refs/heads/TEZ-2003
Commit: c75bb377a51d6f65df3d078ad6f810c625c4812a
Parents: c5d0062
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Jun 1 16:37:26 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../java/org/apache/tez/dag/api/TaskHeartbeatRequest.java | 7 +++++++
.../java/org/apache/tez/dag/api/TaskHeartbeatResponse.java | 8 +++++++-
.../org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java | 8 ++++----
.../java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java | 3 ++-
.../apache/tez/runtime/LogicalIOProcessorRuntimeTask.java | 3 ---
6 files changed, 21 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/c75bb377/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 42c2e1e..55002fe 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -30,5 +30,6 @@ ALL CHANGES:
TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
+ TEZ-2508. rebase 06/01
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/c75bb377/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
index f6bc8f0..b5ff991 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
@@ -29,15 +29,18 @@ public class TaskHeartbeatRequest {
private final TezTaskAttemptID taskAttemptId;
private final List<TezEvent> events;
private final int startIndex;
+ private final int preRoutedStartIndex;
private final int maxEvents;
public TaskHeartbeatRequest(String containerIdentifier, TezTaskAttemptID taskAttemptId, List<TezEvent> events, int startIndex,
+ int preRoutedStartIndex,
int maxEvents) {
this.containerIdentifier = containerIdentifier;
this.taskAttemptId = taskAttemptId;
this.events = events;
this.startIndex = startIndex;
+ this.preRoutedStartIndex = preRoutedStartIndex;
this.maxEvents = maxEvents;
}
@@ -57,6 +60,10 @@ public class TaskHeartbeatRequest {
return startIndex;
}
+ public int getPreRoutedStartIndex() {
+ return preRoutedStartIndex;
+ }
+
public int getMaxEvents() {
return maxEvents;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/c75bb377/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
index b826e76..7f063c4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
@@ -23,12 +23,14 @@ public class TaskHeartbeatResponse {
private final boolean shouldDie;
private final int nextFromEventId;
+ private final int nextPreRoutedEventId;
private final List<TezEvent> events;
- public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events, int nextFromEventId) {
+ public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events, int nextFromEventId, int nextPreRoutedEventId) {
this.shouldDie = shouldDie;
this.events = events;
this.nextFromEventId = nextFromEventId;
+ this.nextPreRoutedEventId = nextPreRoutedEventId;
}
public boolean isShouldDie() {
@@ -42,4 +44,8 @@ public class TaskHeartbeatResponse {
public int getNextFromEventId() {
return nextFromEventId;
}
+
+ public int getNextPreRoutedEventId() {
+ return nextPreRoutedEventId;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/c75bb377/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 1c61a0d..e2d44e2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -79,7 +79,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
protected final TaskHeartbeatHandler taskHeartbeatHandler;
protected final ContainerHeartbeatHandler containerHeartbeatHandler;
- private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null, 0);
+ private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null, 0, 0);
private final ConcurrentMap<TezTaskAttemptID, ContainerId> registeredAttempts =
new ConcurrentHashMap<TezTaskAttemptID, ContainerId>();
@@ -195,7 +195,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
// So - avoiding synchronization.
pingContainerHeartbeatHandler(containerId);
- TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(0, null);
+ TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(0, null, 0);
TezTaskAttemptID taskAttemptID = request.getTaskAttemptId();
if (taskAttemptID != null) {
ContainerId containerIdFromMap = registeredAttempts.get(taskAttemptID);
@@ -241,10 +241,10 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
eventInfo = context
.getCurrentDAG()
.getVertex(taskAttemptID.getTaskID().getVertexID())
- .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(),
+ .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(), request.getPreRoutedStartIndex(),
request.getMaxEvents());
}
- return new TaskHeartbeatResponse(false, eventInfo.getEvents(), eventInfo.getNextFromEventId());
+ return new TaskHeartbeatResponse(false, eventInfo.getEvents(), eventInfo.getNextFromEventId(), eventInfo.getNextPreRoutedFromEventId());
}
public void taskAlive(TezTaskAttemptID taskAttemptId) {
taskHeartbeatHandler.pinged(taskAttemptId);
http://git-wip-us.apache.org/repos/asf/tez/blob/c75bb377/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 3774eb4..83322f2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -363,13 +363,14 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
}
TaskHeartbeatRequest tRequest = new TaskHeartbeatRequest(request.getContainerIdentifier(),
request.getCurrentTaskAttemptID(), request.getEvents(), request.getStartIndex(),
- request.getMaxEvents());
+ request.getPreRoutedStartIndex(), request.getMaxEvents());
tResponse = taskCommunicatorContext.heartbeat(tRequest);
}
TezHeartbeatResponse response = new TezHeartbeatResponse();
response.setLastRequestId(requestId);
response.setEvents(tResponse.getEvents());
response.setNextFromEventId(tResponse.getNextFromEventId());
+ response.setNextPreRoutedEventId(tResponse.getNextPreRoutedEventId());
containerInfo.lastRequestId = requestId;
containerInfo.lastResponse = response;
return response;
http://git-wip-us.apache.org/repos/asf/tez/blob/c75bb377/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
index 449fa0f..c79da5d 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
@@ -172,9 +172,6 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
this.outputsMap = new ConcurrentHashMap<String, LogicalOutput>(numOutputs);
this.outputContextMap = new ConcurrentHashMap<String, OutputContext>(numOutputs);
- this.initializedInputs = new ConcurrentHashMap<String, LogicalInput>();
- this.initializedOutputs = new ConcurrentHashMap<String, LogicalOutput>();
-
this.runInputMap = new LinkedHashMap<String, LogicalInput>();
this.runOutputMap = new LinkedHashMap<String, LogicalOutput>();
[04/51] [abbrv] tez git commit: TEZ-2006. Task communication plane
needs to be pluggable. (sseth)
Posted by ss...@apache.org.
TEZ-2006. Task communication plane needs to be pluggable. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/7ab75d82
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/7ab75d82
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/7ab75d82
Branch: refs/heads/TEZ-2003
Commit: 7ab75d8263b9e6a894b8574656352dc8ef2d56f0
Parents: e091591
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Feb 12 11:25:45 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:22:32 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../apache/tez/dag/api/TaskCommunicator.java | 54 ++
.../tez/dag/api/TaskCommunicatorContext.java | 48 ++
.../tez/dag/api/TaskHeartbeatRequest.java | 63 +++
.../tez/dag/api/TaskHeartbeatResponse.java | 39 ++
.../java/org/apache/tez/dag/app/AppContext.java | 2 +
.../org/apache/tez/dag/app/DAGAppMaster.java | 5 +
.../dag/app/TaskAttemptListenerImpTezDag.java | 532 +++++++------------
.../tez/dag/app/TezTaskCommunicatorImpl.java | 476 +++++++++++++++++
.../app/launcher/LocalContainerLauncher.java | 10 +-
.../tez/dag/app/rm/container/AMContainer.java | 3 +-
.../rm/container/AMContainerEventAssignTA.java | 2 +
.../dag/app/rm/container/AMContainerImpl.java | 1 +
.../apache/tez/dag/app/MockDAGAppMaster.java | 27 +-
.../app/TestTaskAttemptListenerImplTezDag.java | 81 +--
15 files changed, 968 insertions(+), 376 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 1822fcb..d7e4be5 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -1,4 +1,5 @@
ALL CHANGES:
TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration.
+ TEZ-2006. Task communication plane needs to be pluggable.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
new file mode 100644
index 0000000..97f9c16
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tez.dag.api;
+
+import java.net.InetSocketAddress;
+import java.util.Map;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public abstract class TaskCommunicator extends AbstractService {
+ public TaskCommunicator(String name) {
+ super(name);
+ }
+
+ // TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
+ // TODO When talking to an external service, this plugin implementer may need access to a host:port
+ public abstract void registerRunningContainer(ContainerId containerId, String hostname, int port);
+
+ // TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
+ public abstract void registerContainerEnd(ContainerId containerId);
+
+ // TODO TEZ-2003 TaskSpec breakup into a clean interface
+ // TODO TEZ-2003 Add support for priority
+ public abstract void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
+ Map<String, LocalResource> additionalResources,
+ Credentials credentials,
+ boolean credentialsChanged);
+
+ // TODO TEZ-2003 Remove reference to TaskAttemptID
+ public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
+
+ // TODO TEZ-2003 This doesn't necessarily belong here. A server may not start within the AM.
+ public abstract InetSocketAddress getAddress();
+
+ // TODO Eventually. Add methods here to support preemption of tasks.
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
new file mode 100644
index 0000000..9b2d889
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -0,0 +1,48 @@
+/*
+ * 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.tez.dag.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+
+// Do not make calls into this from within a held lock.
+
+// TODO TEZ-2003 Move this into the tez-api module
+public interface TaskCommunicatorContext {
+
+ // TODO TEZ-2003 Add signalling back into this to indicate errors - e.g. Container unregsitered, task no longer running, etc.
+
+ // TODO TEZ-2003 Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
+
+ ApplicationAttemptId getApplicationAttemptId();
+ Credentials getCredentials();
+
+ // TODO TEZ-2003 Move to vertex, taskIndex, version
+ boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
+
+ TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
+
+ boolean isKnownContainer(ContainerId containerId);
+
+ // TODO TEZ-2003 Move to vertex, taskIndex, version
+ void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
+
+ // TODO Eventually Add methods to report availability stats to the scheduler.
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
new file mode 100644
index 0000000..f6bc8f0
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.tez.dag.api;
+
+import java.util.List;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public class TaskHeartbeatRequest {
+
+ // TODO TEZ-2003 Ideally containerIdentifier should not be part of the request.
+ // Replace with a task lookup - vertex name + task index
+ private final String containerIdentifier;
+ // TODO TEZ-2003 Get rid of the task attemptId reference if possible
+ private final TezTaskAttemptID taskAttemptId;
+ private final List<TezEvent> events;
+ private final int startIndex;
+ private final int maxEvents;
+
+
+ public TaskHeartbeatRequest(String containerIdentifier, TezTaskAttemptID taskAttemptId, List<TezEvent> events, int startIndex,
+ int maxEvents) {
+ this.containerIdentifier = containerIdentifier;
+ this.taskAttemptId = taskAttemptId;
+ this.events = events;
+ this.startIndex = startIndex;
+ this.maxEvents = maxEvents;
+ }
+
+ public String getContainerIdentifier() {
+ return containerIdentifier;
+ }
+
+ public TezTaskAttemptID getTaskAttemptId() {
+ return taskAttemptId;
+ }
+
+ public List<TezEvent> getEvents() {
+ return events;
+ }
+
+ public int getStartIndex() {
+ return startIndex;
+ }
+
+ public int getMaxEvents() {
+ return maxEvents;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
new file mode 100644
index 0000000..c82a743
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
@@ -0,0 +1,39 @@
+/*
+ * 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.tez.dag.api;
+
+import java.util.List;
+
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public class TaskHeartbeatResponse {
+
+ private final boolean shouldDie;
+ private List<TezEvent> events;
+
+ public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events) {
+ this.shouldDie = shouldDie;
+ this.events = events;
+ }
+
+ public boolean isShouldDie() {
+ return shouldDie;
+ }
+
+ public List<TezEvent> getEvents() {
+ return events;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
index e909d80..bf3e318 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
@@ -24,6 +24,7 @@ import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -112,4 +113,5 @@ public interface AppContext {
/** Whether the AM is in the process of shutting down/completing */
boolean isAMInCompletionState();
+ Credentials getAppCredentials();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index a45731f..dbcbdd0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -1526,6 +1526,11 @@ public class DAGAppMaster extends AbstractService {
}
@Override
+ public Credentials getAppCredentials() {
+ return amCredentials;
+ }
+
+ @Override
public Map<ApplicationAccessType, String> getApplicationACLs() {
if (getServiceState() != STATE.STARTED) {
throw new TezUncheckedException(
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 15cb801..ff50907 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -18,15 +18,14 @@
package org.apache.tez.dag.app;
import java.io.IOException;
-import java.net.InetAddress;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
import java.net.InetSocketAddress;
import java.net.URISyntaxException;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -37,222 +36,212 @@ import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
import org.apache.tez.runtime.api.impl.EventType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.dag.api.TaskCommunicator;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
+import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
-import org.apache.tez.common.TezConverterUtils;
-import org.apache.tez.common.TezLocalResource;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
+import org.apache.tez.dag.app.rm.TaskSchedulerService;
import org.apache.tez.dag.app.rm.container.AMContainerTask;
-import org.apache.tez.dag.app.security.authorize.TezAMPolicyProvider;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.runtime.api.impl.TezEvent;
-import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
-import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
import org.apache.tez.common.security.JobTokenSecretManager;
-import com.google.common.collect.Maps;
@SuppressWarnings("unchecked")
+@InterfaceAudience.Private
public class TaskAttemptListenerImpTezDag extends AbstractService implements
- TezTaskUmbilicalProtocol, TaskAttemptListener {
-
- private static final ContainerTask TASK_FOR_INVALID_JVM = new ContainerTask(
- null, true, null, null, false);
+ TaskAttemptListener, TaskCommunicatorContext {
private static final Logger LOG = LoggerFactory
.getLogger(TaskAttemptListenerImpTezDag.class);
private final AppContext context;
+ private TaskCommunicator taskCommunicator;
protected final TaskHeartbeatHandler taskHeartbeatHandler;
protected final ContainerHeartbeatHandler containerHeartbeatHandler;
- private final JobTokenSecretManager jobTokenSecretManager;
- private InetSocketAddress address;
- private Server server;
-
- static class ContainerInfo {
- ContainerInfo() {
- this.lastReponse = null;
- this.lastRequestId = 0;
- this.amContainerTask = null;
- this.taskPulled = false;
+
+ private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null);
+
+ private final ConcurrentMap<TezTaskAttemptID, ContainerId> registeredAttempts =
+ new ConcurrentHashMap<TezTaskAttemptID, ContainerId>();
+ private final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
+ new ConcurrentHashMap<ContainerId, ContainerInfo>();
+
+ // Defined primarily to work around ConcurrentMaps not accepting null values
+ private static final class ContainerInfo {
+ TezTaskAttemptID taskAttemptId;
+ ContainerInfo(TezTaskAttemptID taskAttemptId) {
+ this.taskAttemptId = taskAttemptId;
}
- long lastRequestId;
- TezHeartbeatResponse lastReponse;
- AMContainerTask amContainerTask;
- boolean taskPulled;
}
- private ConcurrentMap<TezTaskAttemptID, ContainerId> attemptToInfoMap =
- new ConcurrentHashMap<TezTaskAttemptID, ContainerId>();
+ private static final ContainerInfo NULL_CONTAINER_INFO = new ContainerInfo(null);
- private ConcurrentHashMap<ContainerId, ContainerInfo> registeredContainers =
- new ConcurrentHashMap<ContainerId, ContainerInfo>();
public TaskAttemptListenerImpTezDag(AppContext context,
- TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
- JobTokenSecretManager jobTokenSecretManager) {
+ TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
+ // TODO TEZ-2003 pre-merge. Remove reference to JobTokenSecretManager.
+ JobTokenSecretManager jobTokenSecretManager) {
super(TaskAttemptListenerImpTezDag.class.getName());
this.context = context;
- this.jobTokenSecretManager = jobTokenSecretManager;
this.taskHeartbeatHandler = thh;
this.containerHeartbeatHandler = chh;
+ this.taskCommunicator = new TezTaskCommunicatorImpl(this);
}
@Override
- public void serviceStart() {
- startRpcServer();
- }
-
- protected void startRpcServer() {
- Configuration conf = getConfig();
- if (!conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT)) {
- try {
- server = new RPC.Builder(conf)
- .setProtocol(TezTaskUmbilicalProtocol.class)
- .setBindAddress("0.0.0.0")
- .setPort(0)
- .setInstance(this)
- .setNumHandlers(
- conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
- TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
- .setPortRangeConfig(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE)
- .setSecretManager(jobTokenSecretManager).build();
-
- // Enable service authorization?
- if (conf.getBoolean(
- CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
- false)) {
- refreshServiceAcls(conf, new TezAMPolicyProvider());
- }
-
- server.start();
- InetSocketAddress serverBindAddress = NetUtils.getConnectAddress(server);
- this.address = NetUtils.createSocketAddrForHost(
- serverBindAddress.getAddress().getCanonicalHostName(),
- serverBindAddress.getPort());
-
- LOG.info("Instantiated TaskAttemptListener RPC at " + this.address);
- } catch (IOException e) {
- throw new TezUncheckedException(e);
- }
+ public void serviceInit(Configuration conf) {
+ String taskCommClassName = conf.get(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS);
+ if (taskCommClassName == null) {
+ LOG.info("Using Default Task Communicator");
+ this.taskCommunicator = new TezTaskCommunicatorImpl(this);
} else {
+ LOG.info("Using TaskCommunicator: " + taskCommClassName);
+ Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
+ .getClazz(taskCommClassName);
try {
- this.address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
- } catch (UnknownHostException e) {
+ Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
+ ctor.setAccessible(true);
+ this.taskCommunicator = ctor.newInstance(this);
+ } catch (NoSuchMethodException e) {
+ throw new TezUncheckedException(e);
+ } catch (InvocationTargetException e) {
+ throw new TezUncheckedException(e);
+ } catch (InstantiationException e) {
+ throw new TezUncheckedException(e);
+ } catch (IllegalAccessException e) {
throw new TezUncheckedException(e);
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("Not starting TaskAttemptListener RPC in LocalMode");
}
}
}
- void refreshServiceAcls(Configuration configuration,
- PolicyProvider policyProvider) {
- this.server.refreshServiceAcl(configuration, policyProvider);
+ @Override
+ public void serviceStart() {
+ taskCommunicator.init(getConfig());
+ taskCommunicator.start();
}
@Override
public void serviceStop() {
- stopRpcServer();
- }
-
- protected void stopRpcServer() {
- if (server != null) {
- server.stop();
+ if (taskCommunicator != null) {
+ taskCommunicator.stop();
+ taskCommunicator = null;
}
}
- public InetSocketAddress getAddress() {
- return address;
- }
-
@Override
- public long getProtocolVersion(String protocol, long clientVersion)
- throws IOException {
- return versionID;
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return context.getApplicationAttemptId();
}
@Override
- public ProtocolSignature getProtocolSignature(String protocol,
- long clientVersion, int clientMethodsHash) throws IOException {
- return ProtocolSignature.getProtocolSignature(this, protocol,
- clientVersion, clientMethodsHash);
+ public Credentials getCredentials() {
+ return context.getAppCredentials();
}
@Override
- public ContainerTask getTask(ContainerContext containerContext)
- throws IOException {
+ public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)
+ throws IOException, TezException {
+ ContainerId containerId = ConverterUtils.toContainerId(request
+ .getContainerIdentifier());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Received heartbeat from container"
+ + ", request=" + request);
+ }
- ContainerTask task = null;
+ if (!registeredContainers.containsKey(containerId)) {
+ LOG.warn("Received task heartbeat from unknown container with id: " + containerId +
+ ", asking it to die");
+ return RESPONSE_SHOULD_DIE;
+ }
- if (containerContext == null || containerContext.getContainerIdentifier() == null) {
- LOG.info("Invalid task request with an empty containerContext or containerId");
- task = TASK_FOR_INVALID_JVM;
- } else {
- ContainerId containerId = ConverterUtils.toContainerId(containerContext
- .getContainerIdentifier());
+ // A heartbeat can come in anytime. The AM may have made a decision to kill a running task/container
+ // meanwhile. If the decision is processed through the pipeline before the heartbeat is processed,
+ // the heartbeat will be dropped. Otherwise the heartbeat will be processed - and the system
+ // know how to handle this - via FailedInputEvents for example (relevant only if the heartbeat has events).
+ // So - avoiding synchronization.
+
+ pingContainerHeartbeatHandler(containerId);
+ List<TezEvent> outEvents = null;
+ TezTaskAttemptID taskAttemptID = request.getTaskAttemptId();
+ if (taskAttemptID != null) {
+ ContainerId containerIdFromMap = registeredAttempts.get(taskAttemptID);
+ if (containerIdFromMap == null || !containerIdFromMap.equals(containerId)) {
+ // This can happen when a task heartbeats. Meanwhile the container is unregistered.
+ // The information will eventually make it through to the plugin via a corresponding unregister.
+ // There's a race in that case between the unregister making it through, and this method returning.
+ // TODO TEZ-2003. An exception back is likely a better approach than sending a shouldDie = true,
+ // so that the plugin can handle the scenario. Alternately augment the response with error codes.
+ // Error codes would be better than exceptions.
+ LOG.info("Attempt: " + taskAttemptID + " is not recognized for heartbeats");
+ return RESPONSE_SHOULD_DIE;
+ }
+
+ List<TezEvent> inEvents = request.getEvents();
if (LOG.isDebugEnabled()) {
- LOG.debug("Container with id: " + containerId + " asked for a task");
+ LOG.debug("Ping from " + taskAttemptID.toString() +
+ " events: " + (inEvents != null ? inEvents.size() : -1));
}
- if (!registeredContainers.containsKey(containerId)) {
- if(context.getAllContainers().get(containerId) == null) {
- LOG.info("Container with id: " + containerId
- + " is invalid and will be killed");
- } else {
- LOG.info("Container with id: " + containerId
- + " is valid, but no longer registered, and will be killed");
- }
- task = TASK_FOR_INVALID_JVM;
- } else {
- pingContainerHeartbeatHandler(containerId);
- task = getContainerTask(containerId);
- if (task == null) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("No task current assigned to Container with id: " + containerId);
- }
- } else if (task == TASK_FOR_INVALID_JVM) {
- LOG.info("Container with id: " + containerId
- + " is valid, but no longer registered, and will be killed. Race condition.");
+
+ List<TezEvent> otherEvents = new ArrayList<TezEvent>();
+ for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
+ final EventType eventType = tezEvent.getEventType();
+ if (eventType == EventType.TASK_STATUS_UPDATE_EVENT ||
+ eventType == EventType.TASK_ATTEMPT_COMPLETED_EVENT) {
+ context.getEventHandler()
+ .handle(getTaskAttemptEventFromTezEvent(taskAttemptID, tezEvent));
} else {
- context.getEventHandler().handle(
- new TaskAttemptEventStartedRemotely(task.getTaskSpec()
- .getTaskAttemptID(), containerId, context
- .getApplicationACLs()));
- LOG.info("Container with id: " + containerId + " given task: "
- + task.getTaskSpec().getTaskAttemptID());
+ otherEvents.add(tezEvent);
}
}
+ if(!otherEvents.isEmpty()) {
+ TezVertexID vertexId = taskAttemptID.getTaskID().getVertexID();
+ context.getEventHandler().handle(
+ new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(otherEvents)));
+ }
+ taskHeartbeatHandler.pinged(taskAttemptID);
+ outEvents = context
+ .getCurrentDAG()
+ .getVertex(taskAttemptID.getTaskID().getVertexID())
+ .getTask(taskAttemptID.getTaskID())
+ .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(),
+ request.getMaxEvents());
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("getTask returning task: " + task);
- }
- return task;
+ return new TaskHeartbeatResponse(false, outEvents);
+ }
+
+ @Override
+ public boolean isKnownContainer(ContainerId containerId) {
+ return context.getAllContainers().get(containerId) != null;
+ }
+
+ @Override
+ public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
+ context.getEventHandler().handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
+ pingContainerHeartbeatHandler(containerId);
}
/**
* Child checking whether it can commit.
- *
+ * <p/>
* <br/>
* Repeatedly polls the ApplicationMaster whether it
* {@link Task#canCommit(TezTaskAttemptID)} This is * a legacy from the
@@ -275,25 +264,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
@Override
- public void unregisterTaskAttempt(TezTaskAttemptID attemptId) {
- ContainerId containerId = attemptToInfoMap.get(attemptId);
- if(containerId == null) {
- LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
- return;
- }
- ContainerInfo containerInfo = registeredContainers.get(containerId);
- if(containerInfo == null) {
- LOG.warn("Unregister task attempt: " + attemptId +
- " from non-registered container: " + containerId);
- return;
- }
- synchronized (containerInfo) {
- containerInfo.amContainerTask = null;
- attemptToInfoMap.remove(attemptId);
- }
-
+ public InetSocketAddress getAddress() {
+ return taskCommunicator.getAddress();
}
+ // The TaskAttemptListener register / unregister methods in this class are not thread safe.
+ // The Tez framework should not invoke these methods from multiple threads.
@Override
public void dagComplete(DAG dag) {
// TODO TEZ-2335. Cleanup TaskHeartbeat handler structures.
@@ -313,50 +289,82 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
@Override
public void registerRunningContainer(ContainerId containerId) {
if (LOG.isDebugEnabled()) {
- LOG.debug("ContainerId: " + containerId
- + " registered with TaskAttemptListener");
+ LOG.debug("ContainerId: " + containerId + " registered with TaskAttemptListener");
}
- ContainerInfo oldInfo = registeredContainers.put(containerId, new ContainerInfo());
- if(oldInfo != null) {
+ ContainerInfo oldInfo = registeredContainers.put(containerId, NULL_CONTAINER_INFO);
+ if (oldInfo != null) {
throw new TezUncheckedException(
"Multiple registrations for containerId: " + containerId);
}
+ NodeId nodeId = context.getAllContainers().get(containerId).getContainer().getNodeId();
+ taskCommunicator.registerRunningContainer(containerId, nodeId.getHost(), nodeId.getPort());
+ }
+
+ @Override
+ public void unregisterRunningContainer(ContainerId containerId) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId);
+ }
+ ContainerInfo containerInfo = registeredContainers.remove(containerId);
+ if (containerInfo.taskAttemptId != null) {
+ registeredAttempts.remove(containerInfo.taskAttemptId);
+ }
+ taskCommunicator.registerContainerEnd(containerId);
}
@Override
public void registerTaskAttempt(AMContainerTask amContainerTask,
- ContainerId containerId) {
+ ContainerId containerId) {
ContainerInfo containerInfo = registeredContainers.get(containerId);
- if(containerInfo == null) {
+ if (containerInfo == null) {
throw new TezUncheckedException("Registering task attempt: "
+ amContainerTask.getTask().getTaskAttemptID() + " to unknown container: " + containerId);
}
- synchronized (containerInfo) {
- if(containerInfo.amContainerTask != null) {
- throw new TezUncheckedException("Registering task attempt: "
- + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
- + " with existing assignment to: " + containerInfo.amContainerTask.getTask().getTaskAttemptID());
- }
- containerInfo.amContainerTask = amContainerTask;
- containerInfo.taskPulled = false;
-
- ContainerId containerIdFromMap =
- attemptToInfoMap.put(amContainerTask.getTask().getTaskAttemptID(), containerId);
- if(containerIdFromMap != null) {
- throw new TezUncheckedException("Registering task attempt: "
- + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
- + " when already assigned to: " + containerIdFromMap);
- }
+ if (containerInfo.taskAttemptId != null) {
+ throw new TezUncheckedException("Registering task attempt: "
+ + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+ + " with existing assignment to: " +
+ containerInfo.taskAttemptId);
}
+
+ if (containerInfo.taskAttemptId != null) {
+ throw new TezUncheckedException("Registering task attempt: "
+ + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+ + " with existing assignment to: " +
+ containerInfo.taskAttemptId);
+ }
+
+ // Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
+ registeredContainers.put(containerId, new ContainerInfo(amContainerTask.getTask().getTaskAttemptID()));
+
+ ContainerId containerIdFromMap = registeredAttempts.put(
+ amContainerTask.getTask().getTaskAttemptID(), containerId);
+ if (containerIdFromMap != null) {
+ throw new TezUncheckedException("Registering task attempt: "
+ + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+ + " when already assigned to: " + containerIdFromMap);
+ }
+ taskCommunicator.registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
+ amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
+ amContainerTask.haveCredentialsChanged());
}
@Override
- public void unregisterRunningContainer(ContainerId containerId) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Unregistering Container from TaskAttemptListener: "
- + containerId);
+ public void unregisterTaskAttempt(TezTaskAttemptID attemptId) {
+ ContainerId containerId = registeredAttempts.remove(attemptId);
+ if (containerId == null) {
+ LOG.warn("Unregister task attempt: " + attemptId + " from unknown container");
+ return;
}
- registeredContainers.remove(containerId);
+ ContainerInfo containerInfo = registeredContainers.get(containerId);
+ if (containerInfo == null) {
+ LOG.warn("Unregister task attempt: " + attemptId +
+ " from non-registered container: " + containerId);
+ return;
+ }
+ // Explicitly putting in a new entry so that synchronization is not required on the existing element in the map.
+ registeredContainers.put(containerId, NULL_CONTAINER_INFO);
+ taskCommunicator.unregisterRunningTaskAttempt(attemptId);
}
private void pingContainerHeartbeatHandler(ContainerId containerId) {
@@ -364,7 +372,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
private void pingContainerHeartbeatHandler(TezTaskAttemptID taskAttemptId) {
- ContainerId containerId = attemptToInfoMap.get(taskAttemptId);
+ ContainerId containerId = registeredAttempts.get(taskAttemptId);
if (containerId != null) {
containerHeartbeatHandler.pinged(containerId);
} else {
@@ -373,146 +381,8 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
}
- @Override
- public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
- throws IOException, TezException {
- ContainerId containerId = ConverterUtils.toContainerId(request
- .getContainerIdentifier());
- long requestId = request.getRequestId();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Received heartbeat from container"
- + ", request=" + request);
- }
-
- ContainerInfo containerInfo = registeredContainers.get(containerId);
- if(containerInfo == null) {
- LOG.warn("Received task heartbeat from unknown container with id: " + containerId +
- ", asking it to die");
- TezHeartbeatResponse response = new TezHeartbeatResponse();
- response.setLastRequestId(requestId);
- response.setShouldDie();
- return response;
- }
-
- synchronized (containerInfo) {
- pingContainerHeartbeatHandler(containerId);
-
- if(containerInfo.lastRequestId == requestId) {
- LOG.warn("Old sequenceId received: " + requestId
- + ", Re-sending last response to client");
- return containerInfo.lastReponse;
- }
-
- TezHeartbeatResponse response = new TezHeartbeatResponse();
- response.setLastRequestId(requestId);
-
- TezTaskAttemptID taskAttemptID = request.getCurrentTaskAttemptID();
- if (taskAttemptID != null) {
- ContainerId containerIdFromMap = attemptToInfoMap.get(taskAttemptID);
- if(containerIdFromMap == null || !containerIdFromMap.equals(containerId)) {
- throw new TezException("Attempt " + taskAttemptID
- + " is not recognized for heartbeat");
- }
-
- if(containerInfo.lastRequestId+1 != requestId) {
- throw new TezException("Container " + containerId
- + " has invalid request id. Expected: "
- + containerInfo.lastRequestId+1
- + " and actual: " + requestId);
- }
-
- List<TezEvent> inEvents = request.getEvents();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Ping from " + taskAttemptID.toString() +
- " events: " + (inEvents != null? inEvents.size() : -1));
- }
-
- long currTime = context.getClock().getTime();
- List<TezEvent> otherEvents = new ArrayList<TezEvent>();
- // route TASK_STATUS_UPDATE_EVENT directly to TaskAttempt and route other events
- // (DATA_MOVEMENT_EVENT, TASK_ATTEMPT_COMPLETED_EVENT, TASK_ATTEMPT_FAILED_EVENT)
- // to VertexImpl to ensure the events ordering
- // 1. DataMovementEvent is logged as RecoveryEvent before TaskAttemptFinishedEvent
- // 2. TaskStatusEvent is handled before TaskAttemptFinishedEvent
- for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
- // for now, set the event time on the AM when it is received.
- // this avoids any time disparity between machines.
- tezEvent.setEventReceivedTime(currTime);
- final EventType eventType = tezEvent.getEventType();
- if (eventType == EventType.TASK_STATUS_UPDATE_EVENT) {
- TaskAttemptEvent taskAttemptEvent = new TaskAttemptEventStatusUpdate(taskAttemptID,
- (TaskStatusUpdateEvent) tezEvent.getEvent());
- context.getEventHandler().handle(taskAttemptEvent);
- } else {
- otherEvents.add(tezEvent);
- }
- }
- if(!otherEvents.isEmpty()) {
- TezVertexID vertexId = taskAttemptID.getTaskID().getVertexID();
- context.getEventHandler().handle(
- new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(otherEvents)));
- }
- taskHeartbeatHandler.pinged(taskAttemptID);
- TaskAttemptEventInfo eventInfo = context
- .getCurrentDAG()
- .getVertex(taskAttemptID.getTaskID().getVertexID())
- .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(),
- request.getPreRoutedStartIndex(), request.getMaxEvents());
- response.setEvents(eventInfo.getEvents());
- response.setNextFromEventId(eventInfo.getNextFromEventId());
- response.setNextPreRoutedEventId(eventInfo.getNextPreRoutedFromEventId());
- }
- containerInfo.lastRequestId = requestId;
- containerInfo.lastReponse = response;
- return response;
- }
- }
- private Map<String, TezLocalResource> convertLocalResourceMap(Map<String, LocalResource> ylrs)
- throws IOException {
- Map<String, TezLocalResource> tlrs = Maps.newHashMap();
- if (ylrs != null) {
- for (Entry<String, LocalResource> ylrEntry : ylrs.entrySet()) {
- TezLocalResource tlr;
- try {
- tlr = TezConverterUtils.convertYarnLocalResourceToTez(ylrEntry.getValue());
- } catch (URISyntaxException e) {
- throw new IOException(e);
- }
- tlrs.put(ylrEntry.getKey(), tlr);
- }
- }
- return tlrs;
- }
-
- private ContainerTask getContainerTask(ContainerId containerId) throws IOException {
- ContainerTask containerTask = null;
- ContainerInfo containerInfo = registeredContainers.get(containerId);
- if (containerInfo == null) {
- // This can happen if an unregisterTask comes in after we've done the initial checks for
- // registered containers. (Race between getTask from the container, and a potential STOP_CONTAINER
- // from somewhere within the AM)
- // Implies that an un-registration has taken place and the container needs to be asked to die.
- LOG.info("Container with id: " + containerId
- + " is valid, but no longer registered, and will be killed");
- containerTask = TASK_FOR_INVALID_JVM;
- } else {
- synchronized (containerInfo) {
- if (containerInfo.amContainerTask != null) {
- if (!containerInfo.taskPulled) {
- containerInfo.taskPulled = true;
- AMContainerTask amContainerTask = containerInfo.amContainerTask;
- containerTask = new ContainerTask(amContainerTask.getTask(), false,
- convertLocalResourceMap(amContainerTask.getAdditionalResources()),
- amContainerTask.getCredentials(), amContainerTask.haveCredentialsChanged());
- } else {
- containerTask = null;
- }
- } else {
- containerTask = null;
- }
- }
- }
- return containerTask;
+ public TaskCommunicator getTaskCommunicator() {
+ return taskCommunicator;
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
new file mode 100644
index 0000000..e40f79c
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -0,0 +1,476 @@
+/*
+ * 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.tez.dag.app;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tez.common.*;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.common.security.TokenCache;
+import org.apache.tez.dag.api.TaskCommunicator;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.security.authorize.TezAMPolicyProvider;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+
+@InterfaceAudience.Private
+public class TezTaskCommunicatorImpl extends TaskCommunicator {
+
+ private static final Log LOG = LogFactory.getLog(TezTaskCommunicatorImpl.class);
+
+ private static final ContainerTask TASK_FOR_INVALID_JVM = new ContainerTask(
+ null, true, null, null, false);
+
+ private final TaskCommunicatorContext taskCommunicatorContext;
+
+ private final ConcurrentMap<ContainerId, ContainerInfo> registeredContainers =
+ new ConcurrentHashMap<ContainerId, ContainerInfo>();
+ private final ConcurrentMap<TaskAttempt, ContainerId> attemptToContainerMap =
+ new ConcurrentHashMap<TaskAttempt, ContainerId>();
+
+ private final TezTaskUmbilicalProtocol taskUmbilical;
+ private InetSocketAddress address;
+ private Server server;
+
+ private static final class ContainerInfo {
+
+ ContainerInfo(ContainerId containerId) {
+ this.containerId = containerId;
+ }
+
+ ContainerId containerId;
+ TezHeartbeatResponse lastResponse = null;
+ TaskSpec taskSpec = null;
+ long lastRequestId = 0;
+ Map<String, LocalResource> additionalLRs = null;
+ Credentials credentials = null;
+ boolean credentialsChanged = false;
+ boolean taskPulled = false;
+
+ void reset() {
+ taskSpec = null;
+ additionalLRs = null;
+ credentials = null;
+ credentialsChanged = false;
+ taskPulled = false;
+ }
+ }
+
+
+
+ /**
+ * Construct the service.
+ */
+ public TezTaskCommunicatorImpl(TaskCommunicatorContext taskCommunicatorContext) {
+ super(TezTaskCommunicatorImpl.class.getName());
+ this.taskCommunicatorContext = taskCommunicatorContext;
+ this.taskUmbilical = new TezTaskUmbilicalProtocolImpl();
+ }
+
+
+ @Override
+ public void serviceStart() {
+
+ startRpcServer();
+ }
+
+ @Override
+ public void serviceStop() {
+ stopRpcServer();
+ }
+
+ protected void startRpcServer() {
+ Configuration conf = getConfig();
+ if (!conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT)) {
+ try {
+ JobTokenSecretManager jobTokenSecretManager =
+ new JobTokenSecretManager();
+ Token<JobTokenIdentifier> sessionToken = TokenCache.getSessionToken(taskCommunicatorContext.getCredentials());
+ jobTokenSecretManager.addTokenForJob(
+ taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString(), sessionToken);
+
+ server = new RPC.Builder(conf)
+ .setProtocol(TezTaskUmbilicalProtocol.class)
+ .setBindAddress("0.0.0.0")
+ .setPort(0)
+ .setInstance(taskUmbilical)
+ .setNumHandlers(
+ conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
+ TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
+ .setPortRangeConfig(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE)
+ .setSecretManager(jobTokenSecretManager).build();
+
+ // Enable service authorization?
+ if (conf.getBoolean(
+ CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
+ false)) {
+ refreshServiceAcls(conf, new TezAMPolicyProvider());
+ }
+
+ server.start();
+ this.address = NetUtils.getConnectAddress(server);
+ LOG.info("Instantiated TezTaskCommunicator RPC at " + this.address);
+ } catch (IOException e) {
+ throw new TezUncheckedException(e);
+ }
+ } else {
+ try {
+ this.address = new InetSocketAddress(InetAddress.getLocalHost(), 0);
+ } catch (UnknownHostException e) {
+ throw new TezUncheckedException(e);
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Not starting TaskAttemptListener RPC in LocalMode");
+ }
+ }
+ }
+
+ protected void stopRpcServer() {
+ if (server != null) {
+ server.stop();
+ server = null;
+ }
+ }
+
+ private void refreshServiceAcls(Configuration configuration,
+ PolicyProvider policyProvider) {
+ this.server.refreshServiceAcl(configuration, policyProvider);
+ }
+
+ @Override
+ public void registerRunningContainer(ContainerId containerId, String host, int port) {
+ ContainerInfo oldInfo = registeredContainers.putIfAbsent(containerId, new ContainerInfo(containerId));
+ if (oldInfo != null) {
+ throw new TezUncheckedException("Multiple registrations for containerId: " + containerId);
+ }
+ }
+
+ @Override
+ public void registerContainerEnd(ContainerId containerId) {
+ ContainerInfo containerInfo = registeredContainers.remove(containerId);
+ if (containerInfo != null) {
+ synchronized(containerInfo) {
+ if (containerInfo.taskSpec != null && containerInfo.taskSpec.getTaskAttemptID() != null) {
+ attemptToContainerMap.remove(containerInfo.taskSpec.getTaskAttemptID());
+ }
+ }
+ }
+ }
+
+ @Override
+ public void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
+ Map<String, LocalResource> additionalResources,
+ Credentials credentials, boolean credentialsChanged) {
+
+ ContainerInfo containerInfo = registeredContainers.get(containerId);
+ Preconditions.checkNotNull(containerInfo,
+ "Cannot register task attempt: " + taskSpec.getTaskAttemptID() + " to unknown container: " +
+ containerId);
+ synchronized (containerInfo) {
+ if (containerInfo.taskSpec != null) {
+ throw new TezUncheckedException(
+ "Cannot register task: " + taskSpec.getTaskAttemptID() + " to container: " +
+ containerId + " , with pre-existing assignment: " +
+ containerInfo.taskSpec.getTaskAttemptID());
+ }
+ containerInfo.taskSpec = taskSpec;
+ containerInfo.additionalLRs = additionalResources;
+ containerInfo.credentials = credentials;
+ containerInfo.credentialsChanged = credentialsChanged;
+ containerInfo.taskPulled = false;
+
+ ContainerId oldId = attemptToContainerMap.putIfAbsent(new TaskAttempt(taskSpec.getTaskAttemptID()), containerId);
+ if (oldId != null) {
+ throw new TezUncheckedException(
+ "Attempting to register an already registered taskAttempt with id: " +
+ taskSpec.getTaskAttemptID() + " to containerId: " + containerId +
+ ". Already registered to containerId: " + oldId);
+ }
+ }
+
+ }
+
+ @Override
+ public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID) {
+ TaskAttempt taskAttempt = new TaskAttempt(taskAttemptID);
+ ContainerId containerId = attemptToContainerMap.remove(taskAttempt);
+ if(containerId == null) {
+ LOG.warn("Unregister task attempt: " + taskAttempt + " from unknown container");
+ return;
+ }
+ ContainerInfo containerInfo = registeredContainers.get(containerId);
+ if (containerInfo == null) {
+ LOG.warn("Unregister task attempt: " + taskAttempt +
+ " from non-registered container: " + containerId);
+ return;
+ }
+ synchronized (containerInfo) {
+ containerInfo.reset();
+ attemptToContainerMap.remove(taskAttempt);
+ }
+ }
+
+ @Override
+ public InetSocketAddress getAddress() {
+ return address;
+ }
+
+ public TezTaskUmbilicalProtocol getUmbilical() {
+ return this.taskUmbilical;
+ }
+
+ private class TezTaskUmbilicalProtocolImpl implements TezTaskUmbilicalProtocol {
+
+ @Override
+ public ContainerTask getTask(ContainerContext containerContext) throws IOException {
+ ContainerTask task = null;
+ if (containerContext == null || containerContext.getContainerIdentifier() == null) {
+ LOG.info("Invalid task request with an empty containerContext or containerId");
+ task = TASK_FOR_INVALID_JVM;
+ } else {
+ ContainerId containerId = ConverterUtils.toContainerId(containerContext
+ .getContainerIdentifier());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Container with id: " + containerId + " asked for a task");
+ }
+ task = getContainerTask(containerId);
+ if (task != null && !task.shouldDie()) {
+ taskCommunicatorContext
+ .taskStartedRemotely(task.getTaskSpec().getTaskAttemptID(), containerId);
+ }
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("getTask returning task: " + task);
+ }
+ return task;
+ }
+
+ @Override
+ public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException {
+ return taskCommunicatorContext.canCommit(taskAttemptId);
+ }
+
+ @Override
+ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException,
+ TezException {
+ ContainerId containerId = ConverterUtils.toContainerId(request.getContainerIdentifier());
+ long requestId = request.getRequestId();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Received heartbeat from container"
+ + ", request=" + request);
+ }
+
+ ContainerInfo containerInfo = registeredContainers.get(containerId);
+ if (containerInfo == null) {
+ LOG.warn("Received task heartbeat from unknown container with id: " + containerId +
+ ", asking it to die");
+ TezHeartbeatResponse response = new TezHeartbeatResponse();
+ response.setLastRequestId(requestId);
+ response.setShouldDie();
+ return response;
+ }
+
+ synchronized (containerInfo) {
+ if (containerInfo.lastRequestId == requestId) {
+ LOG.warn("Old sequenceId received: " + requestId
+ + ", Re-sending last response to client");
+ return containerInfo.lastResponse;
+ }
+ }
+
+ TaskHeartbeatResponse tResponse = null;
+
+
+ TezTaskAttemptID taskAttemptID = request.getCurrentTaskAttemptID();
+ if (taskAttemptID != null) {
+ synchronized (containerInfo) {
+ ContainerId containerIdFromMap = attemptToContainerMap.get(new TaskAttempt(taskAttemptID));
+ if (containerIdFromMap == null || !containerIdFromMap.equals(containerId)) {
+ throw new TezException("Attempt " + taskAttemptID
+ + " is not recognized for heartbeat");
+ }
+
+ if (containerInfo.lastRequestId + 1 != requestId) {
+ throw new TezException("Container " + containerId
+ + " has invalid request id. Expected: "
+ + containerInfo.lastRequestId + 1
+ + " and actual: " + requestId);
+ }
+ }
+ TaskHeartbeatRequest tRequest = new TaskHeartbeatRequest(request.getContainerIdentifier(),
+ request.getCurrentTaskAttemptID(), request.getEvents(), request.getStartIndex(),
+ request.getMaxEvents());
+ tResponse = taskCommunicatorContext.heartbeat(tRequest);
+ }
+ TezHeartbeatResponse response;
+ if (tResponse == null) {
+ response = new TezHeartbeatResponse();
+ } else {
+ response = new TezHeartbeatResponse(tResponse.getEvents());
+ }
+ response.setLastRequestId(requestId);
+ containerInfo.lastRequestId = requestId;
+ containerInfo.lastResponse = response;
+ return response;
+ }
+
+
+ // TODO Remove this method once we move to the Protobuf RPC engine
+ @Override
+ public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
+ return versionID;
+ }
+
+ // TODO Remove this method once we move to the Protobuf RPC engine
+ @Override
+ public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
+ int clientMethodsHash) throws IOException {
+ return ProtocolSignature.getProtocolSignature(this, protocol,
+ clientVersion, clientMethodsHash);
+ }
+ }
+
+ private ContainerTask getContainerTask(ContainerId containerId) throws IOException {
+ ContainerInfo containerInfo = registeredContainers.get(containerId);
+ ContainerTask task = null;
+ if (containerInfo == null) {
+ if (taskCommunicatorContext.isKnownContainer(containerId)) {
+ LOG.info("Container with id: " + containerId
+ + " is valid, but no longer registered, and will be killed");
+ } else {
+ LOG.info("Container with id: " + containerId
+ + " is invalid and will be killed");
+ }
+ task = TASK_FOR_INVALID_JVM;
+ } else {
+ synchronized (containerInfo) {
+ if (containerInfo.taskSpec != null) {
+ if (!containerInfo.taskPulled) {
+ containerInfo.taskPulled = true;
+ task = constructContainerTask(containerInfo);
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Task " + containerInfo.taskSpec.getTaskAttemptID() +
+ " already sent to container: " + containerId);
+ }
+ task = null;
+ }
+ } else {
+ task = null;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("No task assigned yet for running container: " + containerId);
+ }
+ }
+ }
+ }
+ return task;
+ }
+
+ private ContainerTask constructContainerTask(ContainerInfo containerInfo) throws IOException {
+ return new ContainerTask(containerInfo.taskSpec, false,
+ convertLocalResourceMap(containerInfo.additionalLRs), containerInfo.credentials,
+ containerInfo.credentialsChanged);
+ }
+
+ private Map<String, TezLocalResource> convertLocalResourceMap(Map<String, LocalResource> ylrs)
+ throws IOException {
+ Map<String, TezLocalResource> tlrs = Maps.newHashMap();
+ if (ylrs != null) {
+ for (Map.Entry<String, LocalResource> ylrEntry : ylrs.entrySet()) {
+ TezLocalResource tlr;
+ try {
+ tlr = TezConverterUtils.convertYarnLocalResourceToTez(ylrEntry.getValue());
+ } catch (URISyntaxException e) {
+ throw new IOException(e);
+ }
+ tlrs.put(ylrEntry.getKey(), tlr);
+ }
+ }
+ return tlrs;
+ }
+
+
+ // Holder for Task information, which eventually will likely be VertexImplm taskIndex, attemptIndex
+ private static class TaskAttempt {
+ // TODO TEZ-2003 Change this to work with VertexName, int id, int version
+ // TODO TEZ-2003 Avoid constructing this unit all over the place
+ private TezTaskAttemptID taskAttemptId;
+
+ TaskAttempt(TezTaskAttemptID taskAttemptId) {
+ this.taskAttemptId = taskAttemptId;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof TaskAttempt)) {
+ return false;
+ }
+
+ TaskAttempt that = (TaskAttempt) o;
+
+ if (!taskAttemptId.equals(that.taskAttemptId)) {
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return taskAttemptId.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "TaskAttempt{" + "taskAttemptId=" + taskAttemptId + '}';
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
index a5cab86..d9d668f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/LocalContainerLauncher.java
@@ -59,6 +59,8 @@ import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskAttemptListenerImpTezDag;
+import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
@@ -86,7 +88,7 @@ public class LocalContainerLauncher extends AbstractService implements
private static final Logger LOG = LoggerFactory.getLogger(LocalContainerLauncher.class);
private final AppContext context;
- private final TaskAttemptListener taskAttemptListener;
+ private final TezTaskUmbilicalProtocol taskUmbilicalProtocol;
private final AtomicBoolean serviceStopped = new AtomicBoolean(false);
private final String workingDirectory;
private final Map<String, String> localEnv = new HashMap<String, String>();
@@ -114,7 +116,9 @@ public class LocalContainerLauncher extends AbstractService implements
String workingDirectory) throws UnknownHostException {
super(LocalContainerLauncher.class.getName());
this.context = context;
- this.taskAttemptListener = taskAttemptListener;
+ TaskAttemptListenerImpTezDag taListener = (TaskAttemptListenerImpTezDag)taskAttemptListener;
+ TezTaskCommunicatorImpl taskComm = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator();
+ this.taskUmbilicalProtocol = taskComm.getUmbilical();
this.workingDirectory = workingDirectory;
AuxiliaryServiceHelper.setServiceDataIntoEnv(
ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID, ByteBuffer.allocate(4).putInt(0), localEnv);
@@ -219,7 +223,7 @@ public class LocalContainerLauncher extends AbstractService implements
tezChild =
createTezChild(context.getAMConf(), event.getContainerId(), tokenIdentifier,
context.getApplicationAttemptId().getAttemptId(), context.getLocalDirs(),
- (TezTaskUmbilicalProtocol) taskAttemptListener,
+ taskUmbilicalProtocol,
TezCommonUtils.parseCredentialsBytes(event.getContainerLaunchContext().getTokens().array()));
} catch (InterruptedException e) {
handleLaunchFailed(e, event.getContainerId());
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
index a6b403d..0fc2e12 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
@@ -22,6 +22,7 @@ import java.util.List;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -32,5 +33,5 @@ public interface AMContainer extends EventHandler<AMContainerEvent>{
public Container getContainer();
public List<TezTaskAttemptID> getAllTaskAttempts();
public TezTaskAttemptID getCurrentTaskAttempt();
-
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
index 682cd02..0398882 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
@@ -27,6 +27,8 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
public class AMContainerEventAssignTA extends AMContainerEvent {
+ // TODO TEZ-2003. Add the task priority to this event.
+
private final TezTaskAttemptID attemptId;
// TODO Maybe have tht TAL pull the remoteTask from the TaskAttempt itself ?
private final TaskSpec remoteTaskSpec;
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index 330f2b7..1acec9c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 8fa57d3..24f3019 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -37,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.tez.dag.app.dag.DAG;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.service.AbstractService;
@@ -50,7 +51,10 @@ import org.apache.tez.client.TezApiVersionInfo;
import org.apache.tez.common.ContainerContext;
import org.apache.tez.common.ContainerTask;
import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.app.launcher.ContainerLauncher;
import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
@@ -72,8 +76,6 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.api.impl.TaskStatistics;
import org.apache.tez.runtime.api.impl.TezEvent;
import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
-import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
@@ -130,6 +132,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
Map<ContainerId, ContainerData> containers = Maps.newConcurrentMap();
ArrayBlockingQueue<Worker> workers;
TaskAttemptListenerImpTezDag taListener;
+ TezTaskCommunicatorImpl taskCommunicator;
AtomicBoolean startScheduling = new AtomicBoolean(true);
AtomicBoolean goFlag;
@@ -194,6 +197,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
@Override
public void serviceStart() throws Exception {
taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
+ taskCommunicator = (TezTaskCommunicatorImpl) taListener.getTaskCommunicator();
eventHandlingThread = new Thread(this);
eventHandlingThread.start();
ExecutorService rawExecutor = Executors.newFixedThreadPool(handlerConcurrency,
@@ -333,10 +337,10 @@ public class MockDAGAppMaster extends DAGAppMaster {
}
}
- private void doHeartbeat(TezHeartbeatRequest request, ContainerData cData) throws Exception {
+ private void doHeartbeat(TaskHeartbeatRequest request, ContainerData cData) throws Exception {
long startTime = System.nanoTime();
long startCpuTime = threadMxBean.getCurrentThreadCpuTime();
- TezHeartbeatResponse response = taListener.heartbeat(request);
+ TaskHeartbeatResponse response = taListener.heartbeat(request);
if (response.shouldDie()) {
cData.remove();
} else {
@@ -388,7 +392,8 @@ public class MockDAGAppMaster extends DAGAppMaster {
try {
if (cData.taId == null) {
// if container is not assigned a task, ask for a task
- ContainerTask cTask = taListener.getTask(new ContainerContext(cData.cIdStr));
+ ContainerTask cTask =
+ taskCommunicator.getUmbilical().getTask(new ContainerContext(cData.cIdStr));
if (cTask != null) {
if (cTask.shouldDie()) {
cData.remove();
@@ -424,8 +429,11 @@ public class MockDAGAppMaster extends DAGAppMaster {
events.add(new TezEvent(new TaskStatusUpdateEvent(counters, progress, stats), new EventMetaData(
EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId),
getContext().getClock().getTime()));
- TezHeartbeatRequest request = new TezHeartbeatRequest(cData.numUpdates, events,
- cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 50000);
+// TezHeartbeatRequest request = new TezHeartbeatRequest(cData.numUpdates, events,
+// cData.cIdStr, cData.taId, cData.nextFromEventId, 50000);
+ TaskHeartbeatRequest request =
+ new TaskHeartbeatRequest(cData.cIdStr, cData.taId, events, cData.nextFromEventId, cData.nextPreRoutedFromEventId,
+ 50000);
doHeartbeat(request, cData);
} else if (version != null && cData.taId.getId() <= version.intValue()) {
preemptContainer(cData);
@@ -436,8 +444,9 @@ public class MockDAGAppMaster extends DAGAppMaster {
new TaskAttemptCompletedEvent(), new EventMetaData(
EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId),
getContext().getClock().getTime()));
- TezHeartbeatRequest request = new TezHeartbeatRequest(++cData.numUpdates, events,
- cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 10000);
+ TaskHeartbeatRequest request =
+ new TaskHeartbeatRequest(cData.cIdStr, cData.taId, events, cData.nextFromEventId, cData.nextPreRoutedFromEventId,
+ 10000);
doHeartbeat(request, cData);
cData.clear();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/7ab75d82/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index d8a7388..c454c7c 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -1,16 +1,16 @@
/*
- * 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.
- */
+* 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.tez.dag.app;
@@ -19,6 +19,7 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
@@ -37,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -45,6 +47,12 @@ import org.apache.tez.common.ContainerTask;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezException;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.Vertex;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
@@ -108,8 +116,18 @@ public class TestTaskAttemptListenerImplTezDag {
doReturn(amContainerMap).when(appContext).getAllContainers();
doReturn(clock).when(appContext).getClock();
- taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+ NodeId nodeId = NodeId.newInstance("localhost", 0);
+ AMContainer amContainer = mock(AMContainer.class);
+ Container container = mock(Container.class);
+ doReturn(nodeId).when(container).getNodeId();
+ doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
+ doReturn(container).when(amContainer).getContainer();
+
+ taskAttemptListener =
+ new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
+ mock(ContainerHeartbeatHandler.class), null);
+ TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+ TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
taskSpec = mock(TaskSpec.class);
doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
@@ -121,32 +139,30 @@ public class TestTaskAttemptListenerImplTezDag {
public void testGetTask() throws IOException {
ContainerId containerId1 = createContainerId(appId, 1);
- doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
- containerTask = taskAttemptListener.getTask(containerContext1);
+ containerTask = tezUmbilical.getTask(containerContext1);
assertTrue(containerTask.shouldDie());
ContainerId containerId2 = createContainerId(appId, 2);
- doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId2);
ContainerContext containerContext2 = new ContainerContext(containerId2.toString());
taskAttemptListener.registerRunningContainer(containerId2);
- containerTask = taskAttemptListener.getTask(containerContext2);
+ containerTask = tezUmbilical.getTask(containerContext2);
assertNull(containerTask);
// Valid task registered
taskAttemptListener.registerTaskAttempt(amContainerTask, containerId2);
- containerTask = taskAttemptListener.getTask(containerContext2);
+ containerTask = tezUmbilical.getTask(containerContext2);
assertFalse(containerTask.shouldDie());
assertEquals(taskSpec, containerTask.getTaskSpec());
// Task unregistered. Should respond to heartbeats
- taskAttemptListener.unregisterTaskAttempt(taskAttemptID);
- containerTask = taskAttemptListener.getTask(containerContext2);
+ taskAttemptListener.unregisterTaskAttempt(taskAttemptId);
+ containerTask = tezUmbilical.getTask(containerContext2);
assertNull(containerTask);
// Container unregistered. Should send a shouldDie = true
taskAttemptListener.unregisterRunningContainer(containerId2);
- containerTask = taskAttemptListener.getTask(containerContext2);
+ containerTask = tezUmbilical.getTask(containerContext2);
assertTrue(containerTask.shouldDie());
ContainerId containerId3 = createContainerId(appId, 3);
@@ -160,27 +176,30 @@ public class TestTaskAttemptListenerImplTezDag {
AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false, 0);
taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3);
taskAttemptListener.unregisterRunningContainer(containerId3);
- containerTask = taskAttemptListener.getTask(containerContext3);
+ containerTask = tezUmbilical.getTask(containerContext3);
assertTrue(containerTask.shouldDie());
}
@Test(timeout = 5000)
public void testGetTaskMultiplePulls() throws IOException {
+ TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+ TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
ContainerId containerId1 = createContainerId(appId, 1);
doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
taskAttemptListener.registerRunningContainer(containerId1);
- containerTask = taskAttemptListener.getTask(containerContext1);
+ containerTask = tezUmbilical.getTask(containerContext1);
assertNull(containerTask);
// Register task
taskAttemptListener.registerTaskAttempt(amContainerTask, containerId1);
- containerTask = taskAttemptListener.getTask(containerContext1);
+ containerTask = tezUmbilical.getTask(containerContext1);
assertFalse(containerTask.shouldDie());
assertEquals(taskSpec, containerTask.getTaskSpec());
// Try pulling again - simulates re-use pull
- containerTask = taskAttemptListener.getTask(containerContext1);
+ containerTask = tezUmbilical.getTask(containerContext1);
assertNull(containerTask);
}
@@ -325,13 +344,11 @@ public class TestTaskAttemptListenerImplTezDag {
return ContainerId.newInstance(appAttemptId, containerIdx);
}
- private static class TaskAttemptListenerImplForTest extends TaskAttemptListenerImpTezDag {
+ private static class TezTaskCommunicatorImplForTest extends TezTaskCommunicatorImpl {
- public TaskAttemptListenerImplForTest(AppContext context,
- TaskHeartbeatHandler thh,
- ContainerHeartbeatHandler chh,
- JobTokenSecretManager jobTokenSecretManager) {
- super(context, thh, chh, jobTokenSecretManager);
+ public TezTaskCommunicatorImplForTest(
+ TaskCommunicatorContext taskCommunicatorContext) {
+ super(taskCommunicatorContext);
}
@Override
[24/51] [abbrv] tez git commit: TEZ-2414.
LogicalIOProcessorRuntimeTask, RuntimeTask,
TezTaskRunner should handle interrupts & carry out necessary cleanups.
Contribtued by Rajesh Balamohan.
Posted by ss...@apache.org.
TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups. Contribtued by Rajesh Balamohan.
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/e4f7ea0a
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/e4f7ea0a
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/e4f7ea0a
Branch: refs/heads/TEZ-2003
Commit: e4f7ea0aa12f9e5b6352e76745898644c530754d
Parents: 9ed22b2
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed May 6 00:39:46 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:34 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../runtime/LogicalIOProcessorRuntimeTask.java | 83 ++++++++++++++++++--
.../org/apache/tez/runtime/RuntimeTask.java | 5 ++
.../apache/tez/runtime/task/TezTaskRunner.java | 71 ++++++++++++++++-
4 files changed, 152 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/e4f7ea0a/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9fc9ed3..f8a71e8 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -20,5 +20,6 @@ ALL CHANGES:
TEZ-2361. Propagate dag completion to TaskCommunicator.
TEZ-2381. Fixes after rebase 04/28.
TEZ-2388. Send dag identifier as part of the fetcher request string.
+ TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/e4f7ea0a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
index 84e5e0d..8263b3f 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
@@ -20,6 +20,9 @@ package org.apache.tez.runtime;
import java.io.Closeable;
import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
import java.nio.ByteBuffer;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
@@ -41,6 +44,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
+import com.google.common.base.Throwables;
import org.apache.commons.lang.exception.ExceptionUtils;
import org.apache.tez.runtime.api.TaskContext;
import org.apache.tez.runtime.api.impl.TezProcessorContextImpl;
@@ -174,6 +178,9 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
this.runInputMap = new LinkedHashMap<String, LogicalInput>();
this.runOutputMap = new LinkedHashMap<String, LogicalOutput>();
+ this.initializedInputs = new ConcurrentHashMap<String, LogicalInput>();
+ this.initializedOutputs = new ConcurrentHashMap<String, LogicalOutput>();
+
this.processorDescriptor = taskSpec.getProcessorDescriptor();
this.serviceConsumerMetadata = serviceConsumerMetadata;
this.envMap = envMap;
@@ -420,6 +427,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
taskSpec.getTaskAttemptID());
initializedInputs.put(edgeName, input);
LOG.info("Initialized Input with src edge: " + edgeName);
+ initializedInputs.put(edgeName, input);
return null;
}
}
@@ -469,6 +477,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
outputContext.getDestinationVertexName(), taskSpec.getTaskAttemptID());
initializedOutputs.put(edgeName, output);
LOG.info("Initialized Output with dest edge: " + edgeName);
+ initializedOutputs.put(edgeName, output);
return null;
}
}
@@ -694,6 +703,13 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
eventsToBeProcessed.addAll(events);
}
+ @Override
+ public synchronized void abortTask() throws Exception {
+ if (processor != null) {
+ processor.abort();
+ }
+ }
+
private void startRouterThread() {
eventRouterThread = new Thread(new RunnableWithNdc() {
public void runInternal() {
@@ -713,6 +729,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
if (!isTaskDone()) {
LOG.warn("Event Router thread interrupted. Returning.");
}
+ Thread.currentThread().interrupt();
return;
}
}
@@ -724,6 +741,12 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
eventRouterThread.start();
}
+ private void maybeResetInterruptStatus() {
+ if (!Thread.currentThread().isInterrupted()) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
private void closeContexts() throws IOException {
closeContext(inputContextMap);
closeContext(outputContextMap);
@@ -763,6 +786,18 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
}
// Close the unclosed IPO
+ /**
+ * Cleanup IPO that are not closed. In case, regular close() has happened in IPO, they
+ * would not be available in the IPOs to be cleaned. So this is safe.
+ *
+ * e.g whenever input gets closed() in normal way, it automatically removes it from
+ * initializedInputs map.
+ *
+ * In case any exception happens in processor close or IO close, it wouldn't be removed from
+ * the initialized IO data structures and here is the chance to close them and release
+ * resources.
+ *
+ */
if (LOG.isDebugEnabled()) {
LOG.debug("Processor closed={}", processorClosed);
LOG.debug("Num of inputs to be closed={}", initializedInputs.size());
@@ -773,10 +808,16 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
try {
processorClosed = true;
processor.close();
- LOG.info("Closed processor for vertex={}, index={}",
+ LOG.info("Closed processor for vertex={}, index={}, interruptedStatus={}",
processor
.getContext().getTaskVertexName(),
- processor.getContext().getTaskVertexIndex());
+ processor.getContext().getTaskVertexIndex(),
+ Thread.currentThread().isInterrupted());
+ maybeResetInterruptStatus();
+ } catch (InterruptedException ie) {
+ //reset the status
+ LOG.info("Resetting interrupt for processor");
+ Thread.currentThread().interrupt();
} catch (Throwable e) {
LOG.warn(
"Ignoring Exception when closing processor(cleanup). Exception class={}, message={}",
@@ -792,13 +833,19 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
inputIterator.remove();
try {
((InputFrameworkInterface)entry.getValue()).close();
+ maybeResetInterruptStatus();
+ } catch (InterruptedException ie) {
+ //reset the status
+ LOG.info("Resetting interrupt status for input with srcVertexName={}",
+ srcVertexName);
+ Thread.currentThread().interrupt();
} catch (Throwable e) {
LOG.warn(
"Ignoring exception when closing input {}(cleanup). Exception class={}, message={}",
srcVertexName, e.getClass().getName(), e.getMessage());
} finally {
- LOG.info("Close input for vertex={}, sourceVertex={}", processor
- .getContext().getTaskVertexName(), srcVertexName);
+ LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
+ .getContext().getTaskVertexName(), srcVertexName, Thread.currentThread().isInterrupted());
}
}
@@ -810,16 +857,26 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
outputIterator.remove();
try {
((OutputFrameworkInterface) entry.getValue()).close();
+ maybeResetInterruptStatus();
+ } catch (InterruptedException ie) {
+ //reset the status
+ LOG.info("Resetting interrupt status for output with destVertexName={}",
+ destVertexName);
+ Thread.currentThread().interrupt();
} catch (Throwable e) {
LOG.warn(
"Ignoring exception when closing output {}(cleanup). Exception class={}, message={}",
destVertexName, e.getClass().getName(), e.getMessage());
} finally {
- LOG.info("Close input for vertex={}, sourceVertex={}", processor
- .getContext().getTaskVertexName(), destVertexName);
+ LOG.info("Close input for vertex={}, sourceVertex={}, interruptedStatus={}", processor
+ .getContext().getTaskVertexName(), destVertexName, Thread.currentThread().isInterrupted());
}
}
+ if (LOG.isDebugEnabled()) {
+ printThreads();
+ }
+
try {
closeContexts();
// Cleanup references which may be held by misbehaved tasks.
@@ -867,6 +924,20 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
inputReadyTracker = null;
objectRegistry = null;
}
+
+
+ /**
+ * Print all threads in JVM (only for debugging)
+ */
+ void printThreads() {
+ //Print the status of all threads in JVM
+ ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
+ long[] threadIds = threadMXBean.getAllThreadIds();
+ for (Long id : threadIds) {
+ ThreadInfo threadInfo = threadMXBean.getThreadInfo(id);
+ LOG.info("ThreadId : " + id + ", name=" + threadInfo.getThreadName());
+ }
+ }
@Private
@VisibleForTesting
http://git-wip-us.apache.org/repos/asf/tez/blob/e4f7ea0a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
index 17d7053..cdfb46a 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
@@ -76,6 +76,10 @@ public abstract class RuntimeTask {
protected final AtomicReference<State> state = new AtomicReference<State>();
+ public boolean isRunning() {
+ return (state.get() == State.RUNNING);
+ }
+
public TezCounters addAndGetTezCounter(String name) {
TezCounters counter = new TezCounters();
counterMap.put(name, counter);
@@ -163,4 +167,5 @@ public abstract class RuntimeTask {
taskDone.set(true);
}
+ public abstract void abortTask() throws Exception;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/e4f7ea0a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
index 33a7f4a..7238d5e 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTaskRunner.java
@@ -25,8 +25,13 @@ import java.security.PrivilegedExceptionAction;
import java.util.Collection;
import java.util.Map;
import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import com.google.common.base.Throwables;
import org.apache.commons.lang.exception.ExceptionUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSError;
@@ -35,6 +40,7 @@ import org.apache.tez.common.CallableWithNdc;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.RuntimeTask;
import org.apache.tez.runtime.api.ExecutionContext;
import org.apache.tez.runtime.api.ObjectRegistry;
import org.apache.tez.runtime.api.impl.EventMetaData;
@@ -61,6 +67,7 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
private final ListeningExecutorService executor;
private volatile ListenableFuture<Void> taskFuture;
private volatile Thread waitingThread;
+ private volatile Thread taskRunner;
private volatile Throwable firstException;
// Effectively a duplicate check, since hadFatalError does the same thing.
@@ -96,7 +103,10 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
taskReporter.registerTask(task, this);
TaskRunnerCallable callable = new TaskRunnerCallable();
Throwable failureCause = null;
- taskFuture = executor.submit(callable);
+ if (!Thread.currentThread().isInterrupted()) {
+ taskFuture = executor.submit(callable);
+ return isShutdownRequested();
+ }
try {
taskFuture.get();
@@ -158,6 +168,10 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
}
}
}
+ return isShutdownRequested();
+ }
+
+ private boolean isShutdownRequested() {
if (shutdownRequested.get()) {
LOG.info("Shutdown requested... returning");
return false;
@@ -173,11 +187,14 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
@Override
public Void run() throws Exception {
try {
+ taskRunner = Thread.currentThread();
LOG.info("Initializing task" + ", taskAttemptId=" + task.getTaskAttemptID());
task.initialize();
if (!Thread.currentThread().isInterrupted() && firstException == null) {
LOG.info("Running task, taskAttemptId=" + task.getTaskAttemptID());
task.run();
+ maybeInterruptWaitingThread();
+
LOG.info("Closing task, taskAttemptId=" + task.getTaskAttemptID());
task.close();
task.setFrameworkCounters();
@@ -199,6 +216,12 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
}
return null;
} catch (Throwable cause) {
+ if (Thread.currentThread().isInterrupted()) {
+ LOG.info("TaskRunnerCallable interrupted=" + Thread.currentThread().isInterrupted()
+ + ", shutdownRequest=" + shutdownRequested.get());
+ Thread.currentThread().interrupt();
+ return null;
+ }
if (cause instanceof FSError) {
// Not immediately fatal, this is an error reported by Hadoop FileSystem
maybeRegisterFirstException(cause);
@@ -255,6 +278,17 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
taskRunning.set(false);
}
}
+
+ private void maybeInterruptWaitingThread() {
+ /**
+ * Possible that the processor is swallowing InterruptException of taskRunner.interrupt().
+ * In such case, interrupt the waitingThread based on the shutdownRequested flag, so that
+ * entire task gets cancelled.
+ */
+ if (shutdownRequested.get()) {
+ waitingThread.interrupt();
+ }
+ }
}
// should wait until all messages are sent to AM before TezChild shutdown
@@ -353,10 +387,43 @@ public class TezTaskRunner implements TezUmbilical, ErrorReporter {
}
}
+ private void abortRunningTask() {
+ if (!taskRunning.get()) {
+ LOG.info("Task is not running");
+ waitingThread.interrupt();
+ return;
+ }
+
+ if (taskRunning.get()) {
+ try {
+ task.abortTask();
+ } catch (Exception e) {
+ LOG.warn("Error when aborting the task", e);
+ try {
+ sendFailure(e, "Error when aborting the task");
+ } catch (Exception ignored) {
+ // Ignored.
+ }
+ }
+ }
+ //Interrupt the relevant threads. TaskRunner should be interrupted preferably.
+ if (isTaskRunning()) {
+ LOG.info("Interrupting taskRunner=" + taskRunner.getName());
+ taskRunner.interrupt();
+ } else {
+ LOG.info("Interrupting waitingThread=" + waitingThread.getName());
+ waitingThread.interrupt();
+ }
+ }
+
+ private boolean isTaskRunning() {
+ return (taskRunning.get() && task.isRunning());
+ }
+
@Override
public void shutdownRequested() {
shutdownRequested.set(true);
- waitingThread.interrupt();
+ abortRunningTask();
}
private String getTaskDiagnosticsString(Throwable t, String message) {
[29/51] [abbrv] tez git commit: TEZ-2471. NPE in
LogicalIOProcessorRuntimeTask while printing thread info. (sseth)
Posted by ss...@apache.org.
TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/a519c295
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/a519c295
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/a519c295
Branch: refs/heads/TEZ-2003
Commit: a519c295ce00f7cf28f0152f5fe8c7d7ced76e03
Parents: db3f6aa
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 22 11:03:11 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java | 5 ++++-
2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/a519c295/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index ca3383c..d651960 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -27,5 +27,6 @@ ALL CHANGES:
TEZ-2434. Allow tasks to be killed in the Runtime.
TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
TEZ-2465. Retrun the status of a kill request in TaskRunner2.
+ TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/a519c295/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
index de08e56..449fa0f 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
@@ -936,7 +936,10 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
long[] threadIds = threadMXBean.getAllThreadIds();
for (Long id : threadIds) {
ThreadInfo threadInfo = threadMXBean.getThreadInfo(id);
- LOG.info("ThreadId : " + id + ", name=" + threadInfo.getThreadName());
+ // The thread could have been shutdown before we read info about it.
+ if (threadInfo != null) {
+ LOG.debug("ThreadId : " + id + ", name=" + threadInfo.getThreadName());
+ }
}
}
[40/51] [abbrv] tez git commit: TEZ-2005. Define basic interface for
pluggable TaskScheduler. (sseth)
Posted by ss...@apache.org.
TEZ-2005. Define basic interface for pluggable TaskScheduler. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/55a9eede
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/55a9eede
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/55a9eede
Branch: refs/heads/TEZ-2003
Commit: 55a9eede6f60efdeaeb4b052daa911d1a790c342
Parents: d799d3b
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Jul 22 22:25:01 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:26:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../tez/common/ContainerSignatureMatcher.java | 64 ++++
.../tez/common/ServicePluginLifecycle.java | 39 ++
.../tez/serviceplugins/api/TaskScheduler.java | 85 +++++
.../api/TaskSchedulerContext.java | 114 ++++++
.../org/apache/tez/common/TezUtilsInternal.java | 1 +
.../tez/dag/api/TaskCommunicatorInterface.java | 18 -
.../org/apache/tez/dag/app/DAGAppMaster.java | 3 +-
.../ServicePluginLifecycleAbstractService.java | 52 +++
.../dag/app/rm/LocalTaskSchedulerService.java | 77 ++--
.../app/rm/TaskSchedulerAppCallbackImpl.java | 89 -----
.../app/rm/TaskSchedulerAppCallbackWrapper.java | 307 ----------------
.../dag/app/rm/TaskSchedulerContextImpl.java | 174 +++++++++
.../app/rm/TaskSchedulerContextImplWrapper.java | 368 +++++++++++++++++++
.../dag/app/rm/TaskSchedulerEventHandler.java | 81 ++--
.../tez/dag/app/rm/TaskSchedulerService.java | 111 ------
.../dag/app/rm/YarnTaskSchedulerService.java | 121 +++---
.../dag/app/rm/container/AMContainerImpl.java | 1 +
.../dag/app/rm/container/AMContainerMap.java | 1 +
.../rm/container/ContainerContextMatcher.java | 1 +
.../rm/container/ContainerSignatureMatcher.java | 60 ---
.../tez/dag/app/rm/TestContainerReuse.java | 148 ++------
.../tez/dag/app/rm/TestLocalTaskScheduler.java | 29 +-
.../app/rm/TestLocalTaskSchedulerService.java | 52 ++-
.../tez/dag/app/rm/TestTaskScheduler.java | 201 +++++-----
.../app/rm/TestTaskSchedulerEventHandler.java | 9 +-
.../dag/app/rm/TestTaskSchedulerHelpers.java | 186 +++++++---
.../rm/TezTestServiceTaskSchedulerService.java | 66 +---
28 files changed, 1357 insertions(+), 1102 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 88dd0c7..a51669d 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -35,5 +35,6 @@ ALL CHANGES:
TEZ-2621. rebase 07/14
TEZ-2124. Change Node tracking to work per external container source.
TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
+ TEZ-2005. Define basic interface for pluggable TaskScheduler.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-api/src/main/java/org/apache/tez/common/ContainerSignatureMatcher.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/ContainerSignatureMatcher.java b/tez-api/src/main/java/org/apache/tez/common/ContainerSignatureMatcher.java
new file mode 100644
index 0000000..c0a1245
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/ContainerSignatureMatcher.java
@@ -0,0 +1,64 @@
+/* Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.tez.common;
+
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface ContainerSignatureMatcher {
+ /**
+ * Checks the compatibility between the specified container signatures.
+ *
+ * @return true if the first signature is a super set of the second
+ * signature.
+ */
+ public boolean isSuperSet(Object cs1, Object cs2);
+
+ /**
+ * Checks if the container signatures match exactly
+ * @return true if exact match
+ */
+ public boolean isExactMatch(Object cs1, Object cs2);
+
+ /**
+ * Gets additional resources specified in lr2, which are not present for lr1
+ *
+ * @param lr1
+ * @param lr2
+ * @return additional resources specified in lr2, which are not present for lr1
+ */
+ public Map<String, LocalResource> getAdditionalResources(Map<String, LocalResource> lr1,
+ Map<String, LocalResource> lr2);
+
+
+ /**
+ * Do a union of 2 signatures
+ * Pre-condition. This function should only be invoked iff cs1 is compatible with cs2.
+ * i.e. isSuperSet should not return false.
+ * @param cs1 Signature 1 Original signature
+ * @param cs2 Signature 2 New signature
+ * @return Union of 2 signatures
+ */
+ public Object union(Object cs1, Object cs2);
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java b/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
new file mode 100644
index 0000000..2eaa7be
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/ServicePluginLifecycle.java
@@ -0,0 +1,39 @@
+/*
+ * 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.tez.common;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface ServicePluginLifecycle {
+
+ /**
+ * Perform any additional initialization which may be required beyond the constructor.
+ */
+ void initialize() throws Exception;
+
+ /**
+ * Start the service. This will be invoked after initialization.
+ */
+ void start() throws Exception;
+
+ /**
+ * Shutdown the service. This will be invoked when the service is shutting down.
+ */
+ void shutdown() throws Exception;
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
new file mode 100644
index 0000000..a5b054f
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskScheduler.java
@@ -0,0 +1,85 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.common.ServicePluginLifecycle;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class TaskScheduler implements ServicePluginLifecycle {
+
+ private final TaskSchedulerContext taskSchedulerContext;
+
+ public TaskScheduler(TaskSchedulerContext taskSchedulerContext) {
+ this.taskSchedulerContext = taskSchedulerContext;
+ }
+
+ @Override
+ public void initialize() throws Exception {
+ }
+
+ @Override
+ public void start() throws Exception {
+ }
+
+ @Override
+ public void shutdown() throws Exception {
+ }
+
+
+ public abstract Resource getAvailableResources();
+
+ public abstract int getClusterNodeCount();
+
+ public abstract void dagComplete();
+
+ public abstract Resource getTotalResources();
+
+ public abstract void blacklistNode(NodeId nodeId);
+
+ public abstract void unblacklistNode(NodeId nodeId);
+
+ public abstract void allocateTask(Object task, Resource capability,
+ String[] hosts, String[] racks, Priority priority,
+ Object containerSignature, Object clientCookie);
+
+ /**
+ * Allocate affinitized to a specific container
+ */
+ public abstract void allocateTask(Object task, Resource capability,
+ ContainerId containerId, Priority priority, Object containerSignature,
+ Object clientCookie);
+
+ /** Plugin writers must ensure to de-allocate a container once it's done, so that it can be collected. */
+ public abstract boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason);
+
+ public abstract Object deallocateContainer(ContainerId containerId);
+
+ public abstract void setShouldUnregister();
+
+ public abstract boolean hasUnregistered();
+
+
+ public final TaskSchedulerContext getContext() {
+ return taskSchedulerContext;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
new file mode 100644
index 0000000..b2c8799
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
@@ -0,0 +1,114 @@
+/*
+ * 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.tez.serviceplugins.api;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.common.ContainerSignatureMatcher;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+
+public interface TaskSchedulerContext {
+
+ public class AppFinalStatus {
+ public final FinalApplicationStatus exitStatus;
+ public final String exitMessage;
+ public final String postCompletionTrackingUrl;
+ public AppFinalStatus(FinalApplicationStatus exitStatus,
+ String exitMessage,
+ String posCompletionTrackingUrl) {
+ this.exitStatus = exitStatus;
+ this.exitMessage = exitMessage;
+ this.postCompletionTrackingUrl = posCompletionTrackingUrl;
+ }
+ }
+
+ enum AMState {
+ IDLE, RUNNING_APP, COMPLETED
+ }
+
+ // TODO Post TEZ-2003. Remove references to YARN constructs like Container, ContainerStatus, NodeReport
+ // upcall to app must be outside locks
+ public void taskAllocated(Object task,
+ Object appCookie,
+ Container container);
+ // this may end up being called for a task+container pair that the app
+ // has not heard about. this can happen because of a race between
+ // taskAllocated() upcall and deallocateTask() downcall
+ public void containerCompleted(Object taskLastAllocated,
+ ContainerStatus containerStatus);
+ public void containerBeingReleased(ContainerId containerId);
+ public void nodesUpdated(List<NodeReport> updatedNodes);
+ public void appShutdownRequested();
+
+ // TODO Post TEZ-2003, this method specifically needs some cleaning up.
+ // ClientAMSecretKey is only relevant when running under YARN. As are ApplicationACLs.
+ public void setApplicationRegistrationData(
+ Resource maxContainerCapability,
+ Map<ApplicationAccessType, String> appAcls,
+ ByteBuffer clientAMSecretKey
+ );
+ public void onError(Throwable t);
+ public float getProgress();
+ public void preemptContainer(ContainerId containerId);
+
+ // TODO Post TEZ-2003. Another method which is primarily relevant to YARN clusters for unregistration.
+ public AppFinalStatus getFinalAppStatus();
+
+
+ // Getters
+
+ // TODO TEZ-2003. To be replaced by getInitialPayload
+ public Configuration getInitialConfiguration();
+
+ public String getAppTrackingUrl();
+
+ /**
+ * A custom cluster identifier allocated to schedulers to generate an AppId, if not making
+ * use of YARN
+ * @return
+ */
+ public long getCustomClusterIdentifier();
+
+ public ContainerSignatureMatcher getContainerSignatureMatcher();
+
+ /**
+ * Get the application attempt id for the running application. Relevant when running under YARN
+ * @return
+ */
+ public ApplicationAttemptId getApplicationAttemptId();
+
+ public String getAppHostName();
+
+ public int getAppClientPort();
+
+ public boolean isSession();
+
+ public AMState getAMState();
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
index 4c8c227..532e83c 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
@@ -45,6 +45,7 @@ import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.records.DAGProtos;
import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
deleted file mode 100644
index 022cd7b..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorInterface.java
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.tez.dag.api;
-
-public interface TaskCommunicatorInterface {
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index d56fb95..ef27ddf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -98,7 +98,6 @@ import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.tez.common.AsyncDispatcher;
import org.apache.tez.common.AsyncDispatcherConcurrent;
import org.apache.tez.common.GcTimeUpdater;
-import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezConverterUtils;
import org.apache.tez.common.TezUtilsInternal;
@@ -149,7 +148,7 @@ import org.apache.tez.dag.app.rm.TaskSchedulerEventHandler;
import org.apache.tez.dag.app.rm.container.AMContainerEventType;
import org.apache.tez.dag.app.rm.container.AMContainerMap;
import org.apache.tez.dag.app.rm.container.ContainerContextMatcher;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.common.ContainerSignatureMatcher;
import org.apache.tez.dag.app.rm.node.AMNodeEventType;
import org.apache.tez.dag.app.rm.node.AMNodeTracker;
import org.apache.tez.dag.app.web.WebUIService;
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/ServicePluginLifecycleAbstractService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/ServicePluginLifecycleAbstractService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/ServicePluginLifecycleAbstractService.java
new file mode 100644
index 0000000..dac1b82
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/ServicePluginLifecycleAbstractService.java
@@ -0,0 +1,52 @@
+/*
+ * 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.tez.dag.app;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.common.ServicePluginLifecycle;
+
+/**
+ * Provides service lifecycle management over ServicePlugins using {@link AbstractService}
+ * @param <T>
+ */
+public class ServicePluginLifecycleAbstractService<T extends ServicePluginLifecycle> extends AbstractService {
+
+ private final T service;
+
+ public ServicePluginLifecycleAbstractService(T service) {
+ super(service.getClass().getName());
+ this.service = service;
+ }
+
+ @Override
+ public void serviceInit(Configuration unused) throws Exception {
+ service.initialize();
+ }
+
+ @Override
+ public void serviceStart() throws Exception {
+ service.start();
+ }
+
+ @Override
+ public void serviceStop() throws Exception {
+ service.shutdown();
+ }
+
+ public T getService() {
+ return service;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
index ef789c5..476d00c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
@@ -20,16 +20,15 @@ package org.apache.tez.dag.app.rm;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
import java.util.concurrent.PriorityBlockingQueue;
-import java.util.concurrent.TimeUnit;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import com.google.common.primitives.Ints;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -43,56 +42,30 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.common.ContainerSignatureMatcher;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-public class LocalTaskSchedulerService extends TaskSchedulerService {
+public class LocalTaskSchedulerService extends TaskScheduler {
private static final Logger LOG = LoggerFactory.getLogger(LocalTaskSchedulerService.class);
- final TaskSchedulerAppCallback realAppClient;
- final TaskSchedulerAppCallback appClientDelegate;
final ContainerSignatureMatcher containerSignatureMatcher;
final PriorityBlockingQueue<TaskRequest> taskRequestQueue;
+ final Configuration conf;
AsyncDelegateRequestHandler taskRequestHandler;
Thread asyncDelegateRequestThread;
- final ExecutorService appCallbackExecutor;
final HashMap<Object, Container> taskAllocations;
- final String appHostName;
- final int appHostPort;
final String appTrackingUrl;
- final AppContext appContext;
final long customContainerAppId;
- public LocalTaskSchedulerService(TaskSchedulerAppCallback appClient,
- ContainerSignatureMatcher containerSignatureMatcher, String appHostName,
- int appHostPort, String appTrackingUrl, long customContainerAppId, AppContext appContext) {
- super(LocalTaskSchedulerService.class.getName());
- this.realAppClient = appClient;
- this.appCallbackExecutor = createAppCallbackExecutorService();
- this.containerSignatureMatcher = containerSignatureMatcher;
- this.appClientDelegate = createAppCallbackDelegate(appClient);
- this.appHostName = appHostName;
- this.appHostPort = appHostPort;
- this.appTrackingUrl = appTrackingUrl;
- this.appContext = appContext;
+ public LocalTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) {
+ super(taskSchedulerContext);
taskRequestQueue = new PriorityBlockingQueue<TaskRequest>();
taskAllocations = new LinkedHashMap<Object, Container>();
- this.customContainerAppId = customContainerAppId;
- }
-
- private ExecutorService createAppCallbackExecutorService() {
- return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
- .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
- }
-
- private TaskSchedulerAppCallback createAppCallbackDelegate(
- TaskSchedulerAppCallback realAppClient) {
- return new TaskSchedulerAppCallbackWrapper(realAppClient,
- appCallbackExecutor);
+ this.appTrackingUrl = taskSchedulerContext.getAppTrackingUrl();
+ this.containerSignatureMatcher = taskSchedulerContext.getContainerSignatureMatcher();
+ this.customContainerAppId = taskSchedulerContext.getCustomClusterIdentifier();
+ this.conf = taskSchedulerContext.getInitialConfiguration();
}
@Override
@@ -160,7 +133,7 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
}
@Override
- public void serviceInit(Configuration conf) {
+ public void initialize() {
taskRequestHandler = createRequestHandler(conf);
asyncDelegateRequestThread = new Thread(taskRequestHandler);
asyncDelegateRequestThread.setDaemon(true);
@@ -168,24 +141,22 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
protected AsyncDelegateRequestHandler createRequestHandler(Configuration conf) {
return new AsyncDelegateRequestHandler(taskRequestQueue,
- new LocalContainerFactory(appContext, customContainerAppId),
+ new LocalContainerFactory(getContext().getApplicationAttemptId(), customContainerAppId),
taskAllocations,
- appClientDelegate,
+ getContext(),
conf);
}
@Override
- public void serviceStart() {
+ public void start() {
asyncDelegateRequestThread.start();
}
@Override
- public void serviceStop() throws InterruptedException {
+ public void shutdown() throws InterruptedException {
if (asyncDelegateRequestThread != null) {
asyncDelegateRequestThread.interrupt();
}
- appCallbackExecutor.shutdownNow();
- appCallbackExecutor.awaitTermination(1000l, TimeUnit.MILLISECONDS);
}
@Override
@@ -202,12 +173,12 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
AtomicInteger nextId;
final ApplicationAttemptId customAppAttemptId;
- public LocalContainerFactory(AppContext appContext, long appIdLong) {
+ public LocalContainerFactory(ApplicationAttemptId appAttemptId, long customAppId) {
this.nextId = new AtomicInteger(1);
ApplicationId appId = ApplicationId
- .newInstance(appIdLong, appContext.getApplicationAttemptId().getApplicationId().getId());
+ .newInstance(customAppId, appAttemptId.getApplicationId().getId());
this.customAppAttemptId = ApplicationAttemptId
- .newInstance(appId, appContext.getApplicationAttemptId().getAttemptId());
+ .newInstance(appId, appAttemptId.getAttemptId());
}
public Container createContainer(Resource capability, Priority priority) {
@@ -330,18 +301,18 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
final BlockingQueue<TaskRequest> taskRequestQueue;
final LocalContainerFactory localContainerFactory;
final HashMap<Object, Container> taskAllocations;
- final TaskSchedulerAppCallback appClientDelegate;
+ final TaskSchedulerContext taskSchedulerContext;
final int MAX_TASKS;
AsyncDelegateRequestHandler(BlockingQueue<TaskRequest> taskRequestQueue,
LocalContainerFactory localContainerFactory,
HashMap<Object, Container> taskAllocations,
- TaskSchedulerAppCallback appClientDelegate,
+ TaskSchedulerContext taskSchedulerContext,
Configuration conf) {
this.taskRequestQueue = taskRequestQueue;
this.localContainerFactory = localContainerFactory;
this.taskAllocations = taskAllocations;
- this.appClientDelegate = appClientDelegate;
+ this.taskSchedulerContext = taskSchedulerContext;
this.MAX_TASKS = conf.getInt(TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS,
TezConfiguration.TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS_DEFAULT);
}
@@ -407,13 +378,13 @@ public class LocalTaskSchedulerService extends TaskSchedulerService {
Container container = localContainerFactory.createContainer(request.capability,
request.priority);
taskAllocations.put(request.task, container);
- appClientDelegate.taskAllocated(request.task, request.clientCookie, container);
+ taskSchedulerContext.taskAllocated(request.task, request.clientCookie, container);
}
void deallocateTask(DeallocateTaskRequest request) {
Container container = taskAllocations.remove(request.task);
if (container != null) {
- appClientDelegate.containerBeingReleased(container.getId());
+ taskSchedulerContext.containerBeingReleased(container.getId());
}
else {
boolean deallocationBeforeAllocation = false;
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
deleted file mode 100644
index ea37e94..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackImpl.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.tez.dag.app.rm;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.Resource;
-
-public class TaskSchedulerAppCallbackImpl implements TaskSchedulerService.TaskSchedulerAppCallback{
-
- private final TaskSchedulerEventHandler tseh;
- private final int schedulerId;
-
- public TaskSchedulerAppCallbackImpl(TaskSchedulerEventHandler tseh, int schedulerId) {
- this.tseh = tseh;
- this.schedulerId = schedulerId;
- }
-
- @Override
- public void taskAllocated(Object task, Object appCookie, Container container) {
- tseh.taskAllocated(schedulerId, task, appCookie, container);
- }
-
- @Override
- public void containerCompleted(Object taskLastAllocated, ContainerStatus containerStatus) {
- tseh.containerCompleted(schedulerId, taskLastAllocated, containerStatus);
- }
-
- @Override
- public void containerBeingReleased(ContainerId containerId) {
- tseh.containerBeingReleased(schedulerId, containerId);
- }
-
- @Override
- public void nodesUpdated(List<NodeReport> updatedNodes) {
- tseh.nodesUpdated(schedulerId, updatedNodes);
- }
-
- @Override
- public void appShutdownRequested() {
- tseh.appShutdownRequested(schedulerId);
- }
-
- @Override
- public void setApplicationRegistrationData(Resource maxContainerCapability,
- Map<ApplicationAccessType, String> appAcls,
- ByteBuffer clientAMSecretKey) {
- tseh.setApplicationRegistrationData(schedulerId, maxContainerCapability, appAcls, clientAMSecretKey);
- }
-
- @Override
- public void onError(Throwable t) {
- tseh.onError(schedulerId, t);
- }
-
- @Override
- public float getProgress() {
- return tseh.getProgress(schedulerId);
- }
-
- @Override
- public void preemptContainer(ContainerId containerId) {
- tseh.preemptContainer(schedulerId, containerId);
- }
-
- @Override
- public AppFinalStatus getFinalAppStatus() {
- return tseh.getFinalAppStatus();
- }
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackWrapper.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackWrapper.java
deleted file mode 100644
index 5de8032..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerAppCallbackWrapper.java
+++ /dev/null
@@ -1,307 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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.tez.dag.app.rm;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
-
-/**
- * Makes use of an ExecutionService to invoke application callbacks. Methods
- * which return values wait for execution to complete - effectively waiting for
- * all previous events in the queue to complete.
- */
-class TaskSchedulerAppCallbackWrapper implements TaskSchedulerAppCallback {
-
- private TaskSchedulerAppCallback real;
-
- ExecutorService executorService;
-
- /**
- * @param real the actual TaskSchedulerAppCallback
- * @param executorService the ExecutorService to be used to send these events.
- */
- public TaskSchedulerAppCallbackWrapper(TaskSchedulerAppCallback real,
- ExecutorService executorService) {
- this.real = real;
- this.executorService = executorService;
- }
-
- @Override
- public void taskAllocated(Object task, Object appCookie, Container container) {
- executorService.submit(new TaskAllocatedCallable(real, task, appCookie,
- container));
- }
-
- @Override
- public void containerCompleted(Object taskLastAllocated,
- ContainerStatus containerStatus) {
- executorService.submit(new ContainerCompletedCallable(real,
- taskLastAllocated, containerStatus));
- }
-
- @Override
- public void containerBeingReleased(ContainerId containerId) {
- executorService
- .submit(new ContainerBeingReleasedCallable(real, containerId));
- }
-
- @Override
- public void nodesUpdated(List<NodeReport> updatedNodes) {
- executorService.submit(new NodesUpdatedCallable(real, updatedNodes));
- }
-
- @Override
- public void appShutdownRequested() {
- executorService.submit(new AppShudownRequestedCallable(real));
- }
-
- @Override
- public void setApplicationRegistrationData(Resource maxContainerCapability,
- Map<ApplicationAccessType, String> appAcls, ByteBuffer key) {
- executorService.submit(new SetApplicationRegistrationDataCallable(real,
- maxContainerCapability, appAcls, key));
- }
-
- @Override
- public void onError(Throwable t) {
- executorService.submit(new OnErrorCallable(real, t));
- }
-
- @Override
- public float getProgress() {
- Future<Float> progressFuture = executorService
- .submit(new GetProgressCallable(real));
- try {
- return progressFuture.get();
- } catch (Exception e) {
- throw new TezUncheckedException(e);
- }
- }
-
- @Override
- public void preemptContainer(ContainerId containerId) {
- executorService.submit(new PreemptContainerCallable(real, containerId));
- }
-
- @Override
- public AppFinalStatus getFinalAppStatus() {
- Future<AppFinalStatus> appFinalStatusFuture = executorService
- .submit(new GetFinalAppStatusCallable(real));
- try {
- return appFinalStatusFuture.get();
- } catch (Exception e) {
- throw new TezUncheckedException(e);
- }
- }
-
-
- static abstract class TaskSchedulerAppCallbackBase {
-
- protected TaskSchedulerAppCallback app;
-
- public TaskSchedulerAppCallbackBase(TaskSchedulerAppCallback app) {
- this.app = app;
- }
- }
-
- static class TaskAllocatedCallable extends TaskSchedulerAppCallbackBase
- implements Callable<Void> {
- private final Object task;
- private final Object appCookie;
- private final Container container;
-
- public TaskAllocatedCallable(TaskSchedulerAppCallback app, Object task,
- Object appCookie, Container container) {
- super(app);
- this.task = task;
- this.appCookie = appCookie;
- this.container = container;
- }
-
- @Override
- public Void call() throws Exception {
- app.taskAllocated(task, appCookie, container);
- return null;
- }
- }
-
- static class ContainerCompletedCallable extends TaskSchedulerAppCallbackBase
- implements Callable<Void> {
-
- private final Object taskLastAllocated;
- private final ContainerStatus containerStatus;
-
- public ContainerCompletedCallable(TaskSchedulerAppCallback app,
- Object taskLastAllocated, ContainerStatus containerStatus) {
- super(app);
- this.taskLastAllocated = taskLastAllocated;
- this.containerStatus = containerStatus;
- }
-
- @Override
- public Void call() throws Exception {
- app.containerCompleted(taskLastAllocated, containerStatus);
- return null;
- }
- }
-
- static class ContainerBeingReleasedCallable extends
- TaskSchedulerAppCallbackBase implements Callable<Void> {
- private final ContainerId containerId;
-
- public ContainerBeingReleasedCallable(TaskSchedulerAppCallback app,
- ContainerId containerId) {
- super(app);
- this.containerId = containerId;
- }
-
- @Override
- public Void call() throws Exception {
- app.containerBeingReleased(containerId);
- return null;
- }
- }
-
- static class NodesUpdatedCallable extends TaskSchedulerAppCallbackBase
- implements Callable<Void> {
- private final List<NodeReport> updatedNodes;
-
- public NodesUpdatedCallable(TaskSchedulerAppCallback app,
- List<NodeReport> updatedNodes) {
- super(app);
- this.updatedNodes = updatedNodes;
- }
-
- @Override
- public Void call() throws Exception {
- app.nodesUpdated(updatedNodes);
- return null;
- }
- }
-
- static class AppShudownRequestedCallable extends TaskSchedulerAppCallbackBase
- implements Callable<Void> {
-
- public AppShudownRequestedCallable(TaskSchedulerAppCallback app) {
- super(app);
- }
-
- @Override
- public Void call() throws Exception {
- app.appShutdownRequested();
- return null;
- }
- }
-
- static class SetApplicationRegistrationDataCallable extends
- TaskSchedulerAppCallbackBase implements Callable<Void> {
-
- private final Resource maxContainerCapability;
- private final Map<ApplicationAccessType, String> appAcls;
- private final ByteBuffer key;
-
- public SetApplicationRegistrationDataCallable(TaskSchedulerAppCallback app,
- Resource maxContainerCapability,
- Map<ApplicationAccessType, String> appAcls,
- ByteBuffer key) {
- super(app);
- this.maxContainerCapability = maxContainerCapability;
- this.appAcls = appAcls;
- this.key = key;
- }
-
- @Override
- public Void call() throws Exception {
- app.setApplicationRegistrationData(maxContainerCapability, appAcls, key);
- return null;
- }
- }
-
- static class OnErrorCallable extends TaskSchedulerAppCallbackBase implements
- Callable<Void> {
-
- private final Throwable throwable;
-
- public OnErrorCallable(TaskSchedulerAppCallback app, Throwable throwable) {
- super(app);
- this.throwable = throwable;
- }
-
- @Override
- public Void call() throws Exception {
- app.onError(throwable);
- return null;
- }
- }
-
- static class PreemptContainerCallable extends TaskSchedulerAppCallbackBase
- implements Callable<Void> {
- private final ContainerId containerId;
-
- public PreemptContainerCallable(TaskSchedulerAppCallback app, ContainerId id) {
- super(app);
- this.containerId = id;
- }
-
- @Override
- public Void call() throws Exception {
- app.preemptContainer(containerId);
- return null;
- }
- }
-
- static class GetProgressCallable extends TaskSchedulerAppCallbackBase
- implements Callable<Float> {
-
- public GetProgressCallable(TaskSchedulerAppCallback app) {
- super(app);
- }
-
- @Override
- public Float call() throws Exception {
- return app.getProgress();
- }
- }
-
- static class GetFinalAppStatusCallable extends TaskSchedulerAppCallbackBase
- implements Callable<AppFinalStatus> {
-
- public GetFinalAppStatusCallable(TaskSchedulerAppCallback app) {
- super(app);
- }
-
- @Override
- public AppFinalStatus call() throws Exception {
- return app.getFinalAppStatus();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
new file mode 100644
index 0000000..890870e
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
@@ -0,0 +1,174 @@
+/*
+ * 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.tez.dag.app.rm;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+
+public class TaskSchedulerContextImpl implements TaskSchedulerContext {
+
+ private final TaskSchedulerEventHandler tseh;
+ private final AppContext appContext;
+ private final int schedulerId;
+ private final String trackingUrl;
+ private final long customClusterIdentifier;
+ private final String appHostName;
+ private final int clientPort;
+ private final Configuration conf;
+
+ public TaskSchedulerContextImpl(TaskSchedulerEventHandler tseh, AppContext appContext,
+ int schedulerId, String trackingUrl, long customClusterIdentifier,
+ String appHostname, int clientPort,
+ Configuration conf) {
+ this.tseh = tseh;
+ this.appContext = appContext;
+ this.schedulerId = schedulerId;
+ this.trackingUrl = trackingUrl;
+ this.customClusterIdentifier = customClusterIdentifier;
+ this.appHostName = appHostname;
+ this.clientPort = clientPort;
+ this.conf = conf;
+
+ }
+
+ @Override
+ public void taskAllocated(Object task, Object appCookie, Container container) {
+ tseh.taskAllocated(schedulerId, task, appCookie, container);
+ }
+
+ @Override
+ public void containerCompleted(Object taskLastAllocated, ContainerStatus containerStatus) {
+ tseh.containerCompleted(schedulerId, taskLastAllocated, containerStatus);
+ }
+
+ @Override
+ public void containerBeingReleased(ContainerId containerId) {
+ tseh.containerBeingReleased(schedulerId, containerId);
+ }
+
+ @Override
+ public void nodesUpdated(List<NodeReport> updatedNodes) {
+ tseh.nodesUpdated(schedulerId, updatedNodes);
+ }
+
+ @Override
+ public void appShutdownRequested() {
+ tseh.appShutdownRequested(schedulerId);
+ }
+
+ @Override
+ public void setApplicationRegistrationData(Resource maxContainerCapability,
+ Map<ApplicationAccessType, String> appAcls,
+ ByteBuffer clientAMSecretKey) {
+ tseh.setApplicationRegistrationData(schedulerId, maxContainerCapability, appAcls, clientAMSecretKey);
+ }
+
+ @Override
+ public void onError(Throwable t) {
+ tseh.onError(schedulerId, t);
+ }
+
+ @Override
+ public float getProgress() {
+ return tseh.getProgress(schedulerId);
+ }
+
+ @Override
+ public void preemptContainer(ContainerId containerId) {
+ tseh.preemptContainer(schedulerId, containerId);
+ }
+
+ @Override
+ public AppFinalStatus getFinalAppStatus() {
+ return tseh.getFinalAppStatus();
+ }
+
+ @Override
+ public Configuration getInitialConfiguration() {
+ return conf;
+ }
+
+
+ @Override
+ public String getAppTrackingUrl() {
+ return trackingUrl;
+ }
+
+ @Override
+ public long getCustomClusterIdentifier() {
+ return customClusterIdentifier;
+ }
+
+ @Override
+ public ContainerSignatureMatcher getContainerSignatureMatcher() {
+ return tseh.getContainerSignatureMatcher();
+ }
+
+ @Override
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return appContext.getApplicationAttemptId();
+ }
+
+ @Override
+ public String getAppHostName() {
+ return appHostName;
+ }
+
+ @Override
+ public int getAppClientPort() {
+ return clientPort;
+ }
+
+ @Override
+ public boolean isSession() {
+ return appContext.isSession();
+ }
+
+ @Override
+ public AMState getAMState() {
+ switch (appContext.getAMState()) {
+
+ case NEW:
+ case INITED:
+ case IDLE:
+ return AMState.IDLE;
+ case RECOVERING:
+ // TODO Is this correct for recovery ?
+ case RUNNING:
+ return AMState.RUNNING_APP;
+ case SUCCEEDED:
+ case FAILED:
+ case KILLED:
+ case ERROR:
+ return AMState.COMPLETED;
+ default:
+ throw new TezUncheckedException("Unexpected state " + appContext.getAMState());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
new file mode 100644
index 0000000..e64ef43
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
@@ -0,0 +1,368 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.tez.dag.app.rm;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+
+/**
+ * Makes use of an ExecutionService to invoke application callbacks. Methods
+ * which return values wait for execution to complete - effectively waiting for
+ * all previous events in the queue to complete.
+ */
+class TaskSchedulerContextImplWrapper implements TaskSchedulerContext {
+
+ private TaskSchedulerContext real;
+
+ private ExecutorService executorService;
+
+ /**
+ * @param real the actual TaskSchedulerAppCallback
+ * @param executorService the ExecutorService to be used to send these events.
+ */
+ public TaskSchedulerContextImplWrapper(TaskSchedulerContext real,
+ ExecutorService executorService) {
+ this.real = real;
+ this.executorService = executorService;
+ }
+
+ @Override
+ public void taskAllocated(Object task, Object appCookie, Container container) {
+ executorService.submit(new TaskAllocatedCallable(real, task, appCookie,
+ container));
+ }
+
+ @Override
+ public void containerCompleted(Object taskLastAllocated,
+ ContainerStatus containerStatus) {
+ executorService.submit(new ContainerCompletedCallable(real,
+ taskLastAllocated, containerStatus));
+ }
+
+ @Override
+ public void containerBeingReleased(ContainerId containerId) {
+ executorService
+ .submit(new ContainerBeingReleasedCallable(real, containerId));
+ }
+
+ @Override
+ public void nodesUpdated(List<NodeReport> updatedNodes) {
+ executorService.submit(new NodesUpdatedCallable(real, updatedNodes));
+ }
+
+ @Override
+ public void appShutdownRequested() {
+ executorService.submit(new AppShudownRequestedCallable(real));
+ }
+
+ @Override
+ public void setApplicationRegistrationData(Resource maxContainerCapability,
+ Map<ApplicationAccessType, String> appAcls, ByteBuffer key) {
+ executorService.submit(new SetApplicationRegistrationDataCallable(real,
+ maxContainerCapability, appAcls, key));
+ }
+
+ @Override
+ public void onError(Throwable t) {
+ executorService.submit(new OnErrorCallable(real, t));
+ }
+
+ @Override
+ public float getProgress() {
+ Future<Float> progressFuture = executorService
+ .submit(new GetProgressCallable(real));
+ try {
+ return progressFuture.get();
+ } catch (Exception e) {
+ throw new TezUncheckedException(e);
+ }
+ }
+
+ @Override
+ public void preemptContainer(ContainerId containerId) {
+ executorService.submit(new PreemptContainerCallable(real, containerId));
+ }
+
+ @Override
+ public AppFinalStatus getFinalAppStatus() {
+ Future<AppFinalStatus> appFinalStatusFuture = executorService
+ .submit(new GetFinalAppStatusCallable(real));
+ try {
+ return appFinalStatusFuture.get();
+ } catch (Exception e) {
+ throw new TezUncheckedException(e);
+ }
+ }
+
+ // Getters which do not need to go through a thread. Underlying implementation
+ // does not use locks.
+
+ @Override
+ public Configuration getInitialConfiguration() {
+ return real.getInitialConfiguration();
+ }
+
+ @Override
+ public String getAppTrackingUrl() {
+ return real.getAppTrackingUrl();
+ }
+
+ @Override
+ public long getCustomClusterIdentifier() {
+ return real.getCustomClusterIdentifier();
+ }
+
+ @Override
+ public ContainerSignatureMatcher getContainerSignatureMatcher() {
+ return real.getContainerSignatureMatcher();
+ }
+
+ @Override
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return real.getApplicationAttemptId();
+ }
+
+ @Override
+ public String getAppHostName() {
+ return real.getAppHostName();
+ }
+
+ @Override
+ public int getAppClientPort() {
+ return real.getAppClientPort();
+ }
+
+ @Override
+ public boolean isSession() {
+ return real.isSession();
+ }
+
+ @Override
+ public AMState getAMState() {
+ return real.getAMState();
+ }
+ // End of getters which do not need to go through a thread. Underlying implementation
+ // does not use locks.
+
+
+ static abstract class TaskSchedulerContextCallbackBase {
+
+ protected TaskSchedulerContext app;
+
+ public TaskSchedulerContextCallbackBase(TaskSchedulerContext app) {
+ this.app = app;
+ }
+ }
+
+ static class TaskAllocatedCallable extends TaskSchedulerContextCallbackBase
+ implements Callable<Void> {
+ private final Object task;
+ private final Object appCookie;
+ private final Container container;
+
+ public TaskAllocatedCallable(TaskSchedulerContext app, Object task,
+ Object appCookie, Container container) {
+ super(app);
+ this.task = task;
+ this.appCookie = appCookie;
+ this.container = container;
+ }
+
+ @Override
+ public Void call() throws Exception {
+ app.taskAllocated(task, appCookie, container);
+ return null;
+ }
+ }
+
+ static class ContainerCompletedCallable extends TaskSchedulerContextCallbackBase
+ implements Callable<Void> {
+
+ private final Object taskLastAllocated;
+ private final ContainerStatus containerStatus;
+
+ public ContainerCompletedCallable(TaskSchedulerContext app,
+ Object taskLastAllocated, ContainerStatus containerStatus) {
+ super(app);
+ this.taskLastAllocated = taskLastAllocated;
+ this.containerStatus = containerStatus;
+ }
+
+ @Override
+ public Void call() throws Exception {
+ app.containerCompleted(taskLastAllocated, containerStatus);
+ return null;
+ }
+ }
+
+ static class ContainerBeingReleasedCallable extends
+ TaskSchedulerContextCallbackBase implements Callable<Void> {
+ private final ContainerId containerId;
+
+ public ContainerBeingReleasedCallable(TaskSchedulerContext app,
+ ContainerId containerId) {
+ super(app);
+ this.containerId = containerId;
+ }
+
+ @Override
+ public Void call() throws Exception {
+ app.containerBeingReleased(containerId);
+ return null;
+ }
+ }
+
+ static class NodesUpdatedCallable extends TaskSchedulerContextCallbackBase
+ implements Callable<Void> {
+ private final List<NodeReport> updatedNodes;
+
+ public NodesUpdatedCallable(TaskSchedulerContext app,
+ List<NodeReport> updatedNodes) {
+ super(app);
+ this.updatedNodes = updatedNodes;
+ }
+
+ @Override
+ public Void call() throws Exception {
+ app.nodesUpdated(updatedNodes);
+ return null;
+ }
+ }
+
+ static class AppShudownRequestedCallable extends TaskSchedulerContextCallbackBase
+ implements Callable<Void> {
+
+ public AppShudownRequestedCallable(TaskSchedulerContext app) {
+ super(app);
+ }
+
+ @Override
+ public Void call() throws Exception {
+ app.appShutdownRequested();
+ return null;
+ }
+ }
+
+ static class SetApplicationRegistrationDataCallable extends
+ TaskSchedulerContextCallbackBase implements Callable<Void> {
+
+ private final Resource maxContainerCapability;
+ private final Map<ApplicationAccessType, String> appAcls;
+ private final ByteBuffer key;
+
+ public SetApplicationRegistrationDataCallable(TaskSchedulerContext app,
+ Resource maxContainerCapability,
+ Map<ApplicationAccessType, String> appAcls,
+ ByteBuffer key) {
+ super(app);
+ this.maxContainerCapability = maxContainerCapability;
+ this.appAcls = appAcls;
+ this.key = key;
+ }
+
+ @Override
+ public Void call() throws Exception {
+ app.setApplicationRegistrationData(maxContainerCapability, appAcls, key);
+ return null;
+ }
+ }
+
+ static class OnErrorCallable extends TaskSchedulerContextCallbackBase implements
+ Callable<Void> {
+
+ private final Throwable throwable;
+
+ public OnErrorCallable(TaskSchedulerContext app, Throwable throwable) {
+ super(app);
+ this.throwable = throwable;
+ }
+
+ @Override
+ public Void call() throws Exception {
+ app.onError(throwable);
+ return null;
+ }
+ }
+
+ static class PreemptContainerCallable extends TaskSchedulerContextCallbackBase
+ implements Callable<Void> {
+ private final ContainerId containerId;
+
+ public PreemptContainerCallable(TaskSchedulerContext app, ContainerId id) {
+ super(app);
+ this.containerId = id;
+ }
+
+ @Override
+ public Void call() throws Exception {
+ app.preemptContainer(containerId);
+ return null;
+ }
+ }
+
+ static class GetProgressCallable extends TaskSchedulerContextCallbackBase
+ implements Callable<Float> {
+
+ public GetProgressCallable(TaskSchedulerContext app) {
+ super(app);
+ }
+
+ @Override
+ public Float call() throws Exception {
+ return app.getProgress();
+ }
+ }
+
+ static class GetFinalAppStatusCallable extends TaskSchedulerContextCallbackBase
+ implements Callable<AppFinalStatus> {
+
+ public GetFinalAppStatusCallable(TaskSchedulerContext app) {
+ super(app);
+ }
+
+ @Override
+ public AppFinalStatus call() throws Exception {
+ return app.getFinalAppStatus();
+ }
+ }
+
+ @VisibleForTesting
+ @InterfaceAudience.Private
+ ExecutorService getExecutorService() {
+ return executorService;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 1ad0059..d8cf080 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -25,11 +25,19 @@ import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.annotations.VisibleForTesting;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback.AppFinalStatus;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
+import org.apache.tez.serviceplugins.api.TaskScheduler;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
+import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@@ -62,7 +70,6 @@ import org.apache.tez.dag.app.dag.event.DAGAppMasterEvent;
import org.apache.tez.dag.app.dag.event.DAGAppMasterEventSchedulingServiceError;
import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType;
import org.apache.tez.dag.app.dag.event.DAGEventSchedulerUpdateTAAssigned;
-import org.apache.tez.dag.app.rm.TaskSchedulerService.TaskSchedulerAppCallback;
import org.apache.tez.dag.app.rm.container.AMContainer;
import org.apache.tez.dag.app.rm.container.AMContainerEventAssignTA;
import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
@@ -70,7 +77,7 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventLaunchRequest;
import org.apache.tez.dag.app.rm.container.AMContainerEventStopRequest;
import org.apache.tez.dag.app.rm.container.AMContainerEventTASucceeded;
import org.apache.tez.dag.app.rm.container.AMContainerState;
-import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.common.ContainerSignatureMatcher;
import org.apache.tez.dag.app.rm.node.AMNodeEventContainerAllocated;
import org.apache.tez.dag.app.rm.node.AMNodeEventNodeCountUpdated;
import org.apache.tez.dag.app.rm.node.AMNodeEventStateChanged;
@@ -106,7 +113,12 @@ public class TaskSchedulerEventHandler extends AbstractService implements
new AtomicBoolean(false);
private final WebUIService webUI;
private final String[] taskSchedulerClasses;
- protected final TaskSchedulerService []taskSchedulers;
+ protected final TaskScheduler[]taskSchedulers;
+ protected final ServicePluginLifecycleAbstractService []taskSchedulerServiceWrappers;
+
+ // Single executor service shared by all Schedulers for context callbacks
+ @VisibleForTesting
+ final ExecutorService appCallbackExecutor;
private final boolean isPureLocalMode;
// If running in non local-only mode, the YARN task scheduler will always run to take care of
@@ -147,6 +159,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
this.webUI = webUI;
this.historyUrl = getHistoryUrl();
this.isPureLocalMode = isPureLocalMode;
+ this.appCallbackExecutor = createAppCallbackExecutorService();
if (this.webUI != null) {
this.webUI.setHistoryUrl(this.historyUrl);
}
@@ -181,7 +194,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
}
}
- taskSchedulers = new TaskSchedulerService[this.taskSchedulerClasses.length];
+ taskSchedulers = new TaskScheduler[this.taskSchedulerClasses.length];
+ taskSchedulerServiceWrappers = new ServicePluginLifecycleAbstractService[this.taskSchedulerClasses.length];
}
public Map<ApplicationAccessType, String> getApplicationAcls() {
@@ -205,6 +219,12 @@ public class TaskSchedulerEventHandler extends AbstractService implements
return taskSchedulers[schedulerId].getTotalResources();
}
+ private ExecutorService createAppCallbackExecutorService() {
+ return Executors.newSingleThreadExecutor(
+ new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d").setDaemon(true)
+ .build());
+ }
+
public synchronized void handleEvent(AMSchedulerEvent sEvent) {
LOG.info("Processing the event " + sEvent.toString());
switch (sEvent.getType()) {
@@ -315,7 +335,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
// stopped.
// AMNodeImpl blacklisting logic does not account for KILLED attempts.
sendEvent(new AMNodeEventTaskAttemptEnded(appContext.getAllContainers().
- get(attemptContainerId).getContainer().getNodeId(), event.getSchedulerId(), attemptContainerId,
+ get(attemptContainerId).getContainer().getNodeId(), event.getSchedulerId(),
+ attemptContainerId,
attempt.getID(), event.getState() == TaskAttemptState.FAILED));
}
}
@@ -389,32 +410,30 @@ public class TaskSchedulerEventHandler extends AbstractService implements
event);
}
- private TaskSchedulerService createTaskScheduler(String host, int port, String trackingUrl,
+ private TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
AppContext appContext,
String schedulerClassName,
long customAppIdIdentifier,
int schedulerId) {
- TaskSchedulerAppCallback appCallback = new TaskSchedulerAppCallbackImpl(this, schedulerId);
+ TaskSchedulerContext rawContext =
+ new TaskSchedulerContextImpl(this, appContext, schedulerId, trackingUrl,
+ customAppIdIdentifier, host, port, getConfig());
+ TaskSchedulerContext wrappedContext = new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
- return new YarnTaskSchedulerService(appCallback, this.containerSignatureMatcher,
- host, port, trackingUrl, appContext);
+ return new YarnTaskSchedulerService(wrappedContext);
} else if (schedulerClassName.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Creating TaskScheduler: Local TaskScheduler");
- return new LocalTaskSchedulerService(appCallback, this.containerSignatureMatcher,
- host, port, trackingUrl, customAppIdIdentifier, appContext);
+ return new LocalTaskSchedulerService(wrappedContext);
} else {
LOG.info("Creating custom TaskScheduler: " + schedulerClassName);
- // TODO TEZ-2003 Temporary reflection with specific parameters. Remove once there is a clean interface.
- Class<? extends TaskSchedulerService> taskSchedulerClazz =
- (Class<? extends TaskSchedulerService>) ReflectionUtils.getClazz(schedulerClassName);
+ Class<? extends TaskScheduler> taskSchedulerClazz =
+ (Class<? extends TaskScheduler>) ReflectionUtils.getClazz(schedulerClassName);
try {
- Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
- .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
- int.class, String.class, long.class, Configuration.class);
+ Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
+ .getConstructor(TaskSchedulerContext.class);
ctor.setAccessible(true);
- return ctor.newInstance(appCallback, appContext, host, port, trackingUrl, customAppIdIdentifier,
- getConfig());
+ return ctor.newInstance(wrappedContext);
} catch (NoSuchMethodException e) {
throw new TezUncheckedException(e);
} catch (InvocationTargetException e) {
@@ -444,6 +463,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
customAppIdIdentifier);
taskSchedulers[i] = createTaskScheduler(host, port,
trackingUrl, appContext, taskSchedulerClasses[i], customAppIdIdentifier, i);
+ taskSchedulerServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskSchedulers[i]);
}
}
@@ -460,8 +480,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
instantiateScheduelrs(serviceAddr.getHostName(), serviceAddr.getPort(), trackingUrl, appContext);
for (int i = 0 ; i < taskSchedulers.length ; i++) {
- taskSchedulers[i].init(getConfig());
- taskSchedulers[i].start();
+ taskSchedulerServiceWrappers[i].init(getConfig());
+ taskSchedulerServiceWrappers[i].start();
if (shouldUnregisterFlag.get()) {
// Flag may have been set earlier when task scheduler was not initialized
// TODO TEZ-2003 Should setRegister / unregister be part of APIs when not YARN specific ?
@@ -510,7 +530,7 @@ public class TaskSchedulerEventHandler extends AbstractService implements
}
@Override
- public void serviceStop() {
+ public void serviceStop() throws InterruptedException {
synchronized(this) {
this.stopEventHandling = true;
if (eventHandlingThread != null)
@@ -518,9 +538,12 @@ public class TaskSchedulerEventHandler extends AbstractService implements
}
for (int i = 0 ; i < taskSchedulers.length ; i++) {
if (taskSchedulers[i] != null) {
- taskSchedulers[i].stop();
+ taskSchedulerServiceWrappers[i].stop();
}
}
+ LOG.info("Shutting down AppCallbackExecutor");
+ appCallbackExecutor.shutdownNow();
+ appCallbackExecutor.awaitTermination(1000l, TimeUnit.MILLISECONDS);
}
// TODO TEZ-2003 Consolidate TaskSchedulerAppCallback methods once these methods are moved into context
@@ -716,6 +739,10 @@ public class TaskSchedulerEventHandler extends AbstractService implements
}
}
+ public ContainerSignatureMatcher getContainerSignatureMatcher() {
+ return containerSignatureMatcher;
+ }
+
public boolean hasUnregistered() {
boolean result = true;
for (int i = 0 ; i < taskSchedulers.length ; i++) {
@@ -757,4 +784,10 @@ public class TaskSchedulerEventHandler extends AbstractService implements
return historyUrl;
}
+
+ @VisibleForTesting
+ @InterfaceAudience.Private
+ ExecutorService getContextExecutorService() {
+ return appCallbackExecutor;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/55a9eede/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
deleted file mode 100644
index 25fd13e..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerService.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements. See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership. The ASF licenses this file
-* to you 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.tez.dag.app.rm;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
-
-public abstract class TaskSchedulerService extends AbstractService{
-
- public TaskSchedulerService(String name) {
- super(name);
- }
-
- public abstract Resource getAvailableResources();
-
- public abstract int getClusterNodeCount();
-
- public abstract void dagComplete();
-
- public abstract Resource getTotalResources();
-
- public abstract void blacklistNode(NodeId nodeId);
-
- public abstract void unblacklistNode(NodeId nodeId);
-
- public abstract void allocateTask(Object task, Resource capability,
- String[] hosts, String[] racks, Priority priority,
- Object containerSignature, Object clientCookie);
-
- /**
- * Allocate affinitized to a specific container
- */
- public abstract void allocateTask(Object task, Resource capability,
- ContainerId containerId, Priority priority, Object containerSignature,
- Object clientCookie);
-
- /** Plugin writers must ensure to de-allocate a container once it's done, so that it can be collected. */
- public abstract boolean deallocateTask(Object task, boolean taskSucceeded, TaskAttemptEndReason endReason);
-
- public abstract Object deallocateContainer(ContainerId containerId);
-
- public abstract void setShouldUnregister();
-
- public abstract boolean hasUnregistered();
-
- public interface TaskSchedulerAppCallback {
- public class AppFinalStatus {
- public final FinalApplicationStatus exitStatus;
- public final String exitMessage;
- public final String postCompletionTrackingUrl;
- public AppFinalStatus(FinalApplicationStatus exitStatus,
- String exitMessage,
- String posCompletionTrackingUrl) {
- this.exitStatus = exitStatus;
- this.exitMessage = exitMessage;
- this.postCompletionTrackingUrl = posCompletionTrackingUrl;
- }
- }
- // upcall to app must be outside locks
- public void taskAllocated(Object task,
- Object appCookie,
- Container container);
- // this may end up being called for a task+container pair that the app
- // has not heard about. this can happen because of a race between
- // taskAllocated() upcall and deallocateTask() downcall
- public void containerCompleted(Object taskLastAllocated,
- ContainerStatus containerStatus);
- public void containerBeingReleased(ContainerId containerId);
- public void nodesUpdated(List<NodeReport> updatedNodes);
- public void appShutdownRequested();
- public void setApplicationRegistrationData(
- Resource maxContainerCapability,
- Map<ApplicationAccessType, String> appAcls,
- ByteBuffer clientAMSecretKey
- );
- public void onError(Throwable t);
- public float getProgress();
- public void preemptContainer(ContainerId containerId);
- public AppFinalStatus getFinalAppStatus();
-
- }
-}
[08/51] [abbrv] tez git commit: TEZ-2283. Fixes after rebase 04/07.
(sseth)
Posted by ss...@apache.org.
TEZ-2283. Fixes after rebase 04/07. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/1f75f327
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/1f75f327
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/1f75f327
Branch: refs/heads/TEZ-2003
Commit: 1f75f327ae00af32c4af85040d09af36dea8b54c
Parents: b3470a1
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:13:30 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:09 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../java/org/apache/tez/dag/app/rm/TestContainerReuse.java | 8 ++++----
.../org/apache/tez/service/impl/ContainerRunnerImpl.java | 4 +---
3 files changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/1f75f327/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index d51686d..6a4399c 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -12,5 +12,6 @@ ALL CHANGES:
TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
TEZ-2241. Miscellaneous fixes after last reabse.
+ TEZ-2283. Fixes after rebase 04/07.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/1f75f327/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 394277a..080c20f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -1242,9 +1242,9 @@ public class TestContainerReuse {
assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
// Task assigned to container completed successfully. Container should be re-used.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta111), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -1254,9 +1254,9 @@ public class TestContainerReuse {
// Task assigned to container completed successfully.
// Verify reuse across hosts.
- taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED));
+ taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
drainableAppCallback.drain();
- verify(taskScheduler).deallocateTask(eq(ta112), eq(true));
+ verify(taskScheduler).deallocateTask(eq(ta112), eq(true), eq((TaskAttemptEndReason)null));
verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
eventHandler.reset();
http://git-wip-us.apache.org/repos/asf/tez/blob/1f75f327/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index e7c02c8..f3fc442 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -69,10 +69,8 @@ import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
import org.apache.tez.runtime.task.TezChild;
import org.apache.tez.runtime.task.TezChild.ContainerExecutionResult;
import org.apache.tez.shufflehandler.ShuffleHandler;
-import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
-import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
import org.apache.tez.util.ProtoConverters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -446,7 +444,7 @@ public class ContainerRunnerImpl extends AbstractService implements ContainerRun
request.getContainerIdString());
taskRunner = new TezTaskRunner(conf, taskUgi, localDirs,
- ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()), umbilical,
+ ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()),
request.getAppAttemptNumber(),
serviceConsumerMetadata, envMap, startedInputsMap, taskReporter, executor, objectRegistry,
pid,
[16/51] [abbrv] tez git commit: TEZ-2302. Allow TaskCommunicators to
subscribe for Vertex updates. (sseth)
Posted by ss...@apache.org.
TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/07e5e428
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/07e5e428
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/07e5e428
Branch: refs/heads/TEZ-2003
Commit: 07e5e42851f0aa4c8727c6ee9adecf639b186fae
Parents: b4ed561
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 9 13:33:48 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:25:10 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../apache/tez/dag/api/TaskCommunicator.java | 20 +++
.../tez/dag/api/TaskCommunicatorContext.java | 14 ++-
.../dag/app/TaskAttemptListenerImpTezDag.java | 52 +++-----
.../dag/app/TaskCommunicatorContextImpl.java | 124 +++++++++++++++++++
.../tez/dag/app/TezTaskCommunicatorImpl.java | 6 +
.../java/org/apache/tez/dag/app/dag/DAG.java | 2 +
.../apache/tez/dag/app/dag/impl/DAGImpl.java | 5 +
8 files changed, 188 insertions(+), 36 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/07e5e428/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index 9d6b220..ca5225e 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -15,5 +15,6 @@ ALL CHANGES:
TEZ-2283. Fixes after rebase 04/07.
TEZ-2284. Separate TaskReporter into an interface.
TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
+ TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/07e5e428/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
index 945091e..a2cd858 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.api.impl.TaskSpec;
@@ -54,4 +55,23 @@ public abstract class TaskCommunicator extends AbstractService {
public abstract InetSocketAddress getAddress();
// TODO Eventually. Add methods here to support preemption of tasks.
+
+ /**
+ * Receive notifications on vertex state changes.
+ * <p/>
+ * State changes will be received based on the registration via {@link
+ * org.apache.tez.runtime.api.InputInitializerContext#registerForVertexStateUpdates(String,
+ * java.util.Set)}. Notifications will be received for all registered state changes, and not just
+ * for the latest state update. They will be in order in which the state change occurred. </p>
+ *
+ * Extensive processing should not be performed via this method call. Instead this should just be
+ * used as a notification mechanism.
+ * <br>This method may be invoked concurrently with other invocations into the TaskCommunicator and
+ * multi-threading/concurrency implications must be considered.
+ * @param stateUpdate an event indicating the name of the vertex, and it's updated state.
+ * Additional information may be available for specific events, Look at the
+ * type hierarchy for {@link org.apache.tez.dag.api.event.VertexStateUpdate}
+ * @throws Exception
+ */
+ public abstract void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/07e5e428/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
index 0c3bac3..19caed9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -16,10 +16,12 @@ package org.apache.tez.dag.api;
import javax.annotation.Nullable;
import java.io.IOException;
+import java.util.Set;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.event.VertexState;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -48,7 +50,7 @@ public interface TaskCommunicatorContext {
void containerAlive(ContainerId containerId);
// TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
- void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
+ void taskStartedRemotely(TezTaskAttemptID taskAttemptId, ContainerId containerId);
// TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
@@ -56,6 +58,16 @@ public interface TaskCommunicatorContext {
// TODO TEZ-2003 Move to vertex, taskIndex, version. Rename to taskAttempt*
void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason, @Nullable String diagnostics);
+ /**
+ * Register to get notifications on updates to the specified vertex. Notifications will be sent
+ * via {@link org.apache.tez.runtime.api.InputInitializer#onVertexStateUpdated(org.apache.tez.dag.api.event.VertexStateUpdate)} </p>
+ *
+ * This method can only be invoked once. Duplicate invocations will result in an error.
+ *
+ * @param vertexName the vertex name for which notifications are required.
+ * @param stateSet the set of states for which notifications are required. null implies all
+ */
+ void registerForVertexStateUpdates(String vertexName, @Nullable Set<VertexState> stateSet);
// TODO TEZ-2003 API. Should a method exist for task succeeded.
// TODO Eventually Add methods to report availability stats to the scheduler.
http://git-wip-us.apache.org/repos/asf/tez/blob/07e5e428/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index a6994d2..386e4af 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -17,6 +17,7 @@
package org.apache.tez.dag.app;
+import javax.annotation.Nullable;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
@@ -26,6 +27,7 @@ import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -34,13 +36,13 @@ import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
import org.apache.tez.runtime.api.impl.EventType;
+import com.google.common.base.Preconditions;
+import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.common.TezUtilsInternal;
@@ -66,14 +68,12 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.runtime.api.impl.TezEvent;
import org.apache.tez.common.security.JobTokenSecretManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
@SuppressWarnings("unchecked")
@InterfaceAudience.Private
public class TaskAttemptListenerImpTezDag extends AbstractService implements
- TaskAttemptListener, TaskCommunicatorContext {
+ TaskAttemptListener {
private static final Logger LOG = LoggerFactory
.getLogger(TaskAttemptListenerImpTezDag.class);
@@ -123,7 +123,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
- taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i]);
+ taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
}
// TODO TEZ-2118 Start using taskCommunicator indices properly
}
@@ -144,13 +144,13 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
}
- private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier) {
+ private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
LOG.info("Using Default Task Communicator");
- return new TezTaskCommunicatorImpl(this);
+ return new TezTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
} else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
LOG.info("Using Default Local Task Communicator");
- return new TezLocalTaskCommunicatorImpl(this);
+ return new TezLocalTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
} else {
LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
@@ -158,7 +158,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
try {
Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
ctor.setAccessible(true);
- return ctor.newInstance(this);
+ return ctor.newInstance(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
} catch (NoSuchMethodException e) {
throw new TezUncheckedException(e);
} catch (InvocationTargetException e) {
@@ -170,18 +170,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
}
}
-
- @Override
- public ApplicationAttemptId getApplicationAttemptId() {
- return context.getApplicationAttemptId();
- }
-
- @Override
- public Credentials getCredentials() {
- return context.getAppCredentials();
- }
-
- @Override
public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)
throws IOException, TezException {
ContainerId containerId = ConverterUtils.toContainerId(request
@@ -251,30 +239,20 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
return new TaskHeartbeatResponse(false, outEvents);
}
-
- @Override
- public boolean isKnownContainer(ContainerId containerId) {
- return context.getAllContainers().get(containerId) != null;
- }
-
- @Override
public void taskAlive(TezTaskAttemptID taskAttemptId) {
taskHeartbeatHandler.pinged(taskAttemptId);
}
- @Override
public void containerAlive(ContainerId containerId) {
pingContainerHeartbeatHandler(containerId);
}
- @Override
public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
context.getEventHandler()
.handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));
pingContainerHeartbeatHandler(containerId);
}
- @Override
public void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
String diagnostics) {
// Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
@@ -287,7 +265,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
taskAttemptEndReason)));
}
- @Override
public void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
String diagnostics) {
// Regular flow via TaskAttempt will take care of un-registering from the heartbeat handler,
@@ -300,6 +277,11 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
taskAttemptEndReason)));
}
+ public void vertexStateUpdateNotificationReceived(VertexStateUpdate event, int taskCommIndex) throws
+ Exception {
+ taskCommunicators[taskCommIndex].onVertexStateUpdated(event);
+ }
+
/**
* Child checking whether it can commit.
@@ -309,7 +291,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
* {@link Task#canCommit(TezTaskAttemptID)} This is * a legacy from the
* centralized commit protocol handling by the JobTracker.
*/
- @Override
+// @Override
public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException {
LOG.info("Commit go/no-go request from " + taskAttemptId.toString());
// An attempt is asking if it can commit its output. This can be decided
http://git-wip-us.apache.org/repos/asf/tez/blob/07e5e428/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
new file mode 100644
index 0000000..3714c3c
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -0,0 +1,124 @@
+/*
+ * 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.tez.dag.app;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.app.dag.VertexStateUpdateListener;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+@InterfaceAudience.Private
+public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, VertexStateUpdateListener {
+
+
+ private final AppContext context;
+ private final TaskAttemptListenerImpTezDag taskAttemptListener;
+ private final int taskCommunicatorIndex;
+
+ public TaskCommunicatorContextImpl(AppContext appContext,
+ TaskAttemptListenerImpTezDag taskAttemptListener,
+ int taskCommunicatorIndex) {
+ this.context = appContext;
+ this.taskAttemptListener = taskAttemptListener;
+ this.taskCommunicatorIndex = taskCommunicatorIndex;
+ }
+
+ @Override
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return context.getApplicationAttemptId();
+ }
+
+ @Override
+ public Credentials getCredentials() {
+ return context.getAppCredentials();
+ }
+
+ @Override
+ public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException {
+ return taskAttemptListener.canCommit(taskAttemptId);
+ }
+
+ @Override
+ public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException,
+ TezException {
+ return taskAttemptListener.heartbeat(request);
+ }
+
+ @Override
+ public boolean isKnownContainer(ContainerId containerId) {
+ return context.getAllContainers().get(containerId) != null;
+ }
+
+ @Override
+ public void taskAlive(TezTaskAttemptID taskAttemptId) {
+ taskAttemptListener.taskAlive(taskAttemptId);
+ }
+
+ @Override
+ public void containerAlive(ContainerId containerId) {
+ taskAttemptListener.containerAlive(containerId);
+ }
+
+ @Override
+ public void taskStartedRemotely(TezTaskAttemptID taskAttemptId, ContainerId containerId) {
+ taskAttemptListener.taskStartedRemotely(taskAttemptId, containerId);
+ }
+
+ @Override
+ public void taskKilled(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+ @Nullable String diagnostics) {
+ taskAttemptListener.taskKilled(taskAttemptId, taskAttemptEndReason, diagnostics);
+ }
+
+ @Override
+ public void taskFailed(TezTaskAttemptID taskAttemptId, TaskAttemptEndReason taskAttemptEndReason,
+ @Nullable String diagnostics) {
+ taskAttemptListener.taskFailed(taskAttemptId, taskAttemptEndReason, diagnostics);
+
+ }
+
+ @Override
+ public void registerForVertexStateUpdates(String vertexName,
+ @Nullable Set<VertexState> stateSet) {
+ Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
+ context.getCurrentDAG().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet, this);
+ }
+
+
+ @Override
+ public void onStateUpdated(VertexStateUpdate event) {
+ try {
+ taskAttemptListener.vertexStateUpdateNotificationReceived(event, taskCommunicatorIndex);
+ } catch (Exception e) {
+ // TODO TEZ-2003 This needs to be propagated to the DAG as a user error.
+ throw new TezUncheckedException(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/07e5e428/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index a4a707b..fa2749a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -48,6 +48,7 @@ import org.apache.tez.dag.api.TaskHeartbeatResponse;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
import org.apache.tez.dag.app.security.authorize.TezAMPolicyProvider;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.runtime.api.impl.TaskSpec;
@@ -254,6 +255,11 @@ public class TezTaskCommunicatorImpl extends TaskCommunicator {
return address;
}
+ @Override
+ public void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception {
+ // Empty. Not registering, or expecting any updates.
+ }
+
protected String getTokenIdentifier() {
return tokenIdentifier;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/07e5e428/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
index 4c3426a..6d6872b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
@@ -94,4 +94,6 @@ public interface DAG {
Map<String, TezVertexID> getVertexNameIDMapping();
+ StateChangeNotifier getStateChangeNotifier();
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/07e5e428/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index 3d44ba6..ef2df78 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -702,6 +702,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
}
@Override
+ public StateChangeNotifier getStateChangeNotifier() {
+ return entityUpdateTracker;
+ }
+
+ @Override
public TezCounters getAllCounters() {
readLock.lock();
[50/51] [abbrv] tez git commit: TEZ-2126. Add unit tests for
verifying multiple schedulers, launchers, communicators. (sseth)
Posted by ss...@apache.org.
TEZ-2126. Add unit tests for verifying multiple schedulers, launchers,
communicators. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/4a21040a
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/4a21040a
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/4a21040a
Branch: refs/heads/TEZ-2003
Commit: 4a21040afe06342588dd54511b1c703abe617a6e
Parents: 4766b71
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Aug 6 01:04:31 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Aug 6 01:27:52 2015 -0700
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 1 +
.../tez/dag/api/NamedEntityDescriptor.java | 7 +
.../org/apache/tez/dag/app/DAGAppMaster.java | 163 ++++----
.../dag/app/TaskAttemptListenerImpTezDag.java | 94 ++---
.../apache/tez/dag/app/dag/impl/VertexImpl.java | 9 +-
.../app/launcher/ContainerLauncherRouter.java | 126 ++++---
.../dag/app/rm/TaskSchedulerEventHandler.java | 137 +++----
.../apache/tez/dag/app/MockDAGAppMaster.java | 3 +-
.../apache/tez/dag/app/TestDAGAppMaster.java | 300 +++++++++++++++
.../app/TestTaskAttemptListenerImplTezDag.java | 44 ++-
.../app/TestTaskAttemptListenerImplTezDag2.java | 6 +-
.../dag/app/TestTaskCommunicatorManager.java | 369 +++++++++++++++++++
.../tez/dag/app/dag/impl/TestVertexImpl2.java | 279 ++++++++++++--
.../launcher/TestContainerLauncherRouter.java | 361 ++++++++++++++++++
.../app/rm/TestTaskSchedulerEventHandler.java | 330 ++++++++++++++++-
.../dag/app/rm/TestTaskSchedulerHelpers.java | 4 +-
16 files changed, 1907 insertions(+), 326 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
index c7a3dcc..f921739 100644
--- a/TEZ-2003-CHANGES.txt
+++ b/TEZ-2003-CHANGES.txt
@@ -42,5 +42,6 @@ ALL CHANGES:
TEZ-2441. Add tests for TezTaskRunner2.
TEZ-2657. Add tests for client side changes - specifying plugins, etc.
TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs.
+ TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators.
INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
index 723d43f..17c8c6c 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java
@@ -35,4 +35,11 @@ public class NamedEntityDescriptor<T extends NamedEntityDescriptor<T>> extends E
super.setUserPayload(userPayload);
return (T) this;
}
+
+ @Override
+ public String toString() {
+ boolean hasPayload =
+ getUserPayload() == null ? false : getUserPayload().getPayload() == null ? false : true;
+ return "EntityName=" + entityName + ", ClassName=" + getClassName() + ", hasPayload=" + hasPayload;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 9b16a90..ed4f520 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -59,6 +59,7 @@ import java.util.regex.Pattern;
import com.google.common.collect.BiMap;
import com.google.common.collect.HashBiMap;
+import com.google.common.collect.Lists;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.Options;
@@ -389,42 +390,16 @@ public class DAGAppMaster extends AbstractService {
this.isLocal = conf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
- List<NamedEntityDescriptor> taskSchedulerDescriptors;
- List<NamedEntityDescriptor> containerLauncherDescriptors;
- List<NamedEntityDescriptor> taskCommunicatorDescriptors;
- boolean tezYarnEnabled = true;
- boolean uberEnabled = false;
-
- if (!isLocal) {
- if (amPluginDescriptorProto == null) {
- tezYarnEnabled = true;
- uberEnabled = false;
- } else {
- tezYarnEnabled = amPluginDescriptorProto.getContainersEnabled();
- uberEnabled = amPluginDescriptorProto.getUberEnabled();
- }
- } else {
- tezYarnEnabled = false;
- uberEnabled = true;
- }
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(amConf);
- taskSchedulerDescriptors = parsePlugin(taskSchedulers,
- (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskSchedulersCount() == 0 ?
- null :
- amPluginDescriptorProto.getTaskSchedulersList()),
- tezYarnEnabled, uberEnabled);
+ List<NamedEntityDescriptor> taskSchedulerDescriptors = Lists.newLinkedList();
+ List<NamedEntityDescriptor> containerLauncherDescriptors = Lists.newLinkedList();
+ List<NamedEntityDescriptor> taskCommunicatorDescriptors = Lists.newLinkedList();
- containerLauncherDescriptors = parsePlugin(containerLaunchers,
- (amPluginDescriptorProto == null ||
- amPluginDescriptorProto.getContainerLaunchersCount() == 0 ? null :
- amPluginDescriptorProto.getContainerLaunchersList()),
- tezYarnEnabled, uberEnabled);
+ parseAllPlugins(taskSchedulerDescriptors, taskSchedulers, containerLauncherDescriptors,
+ containerLaunchers, taskCommunicatorDescriptors, taskCommunicators, amPluginDescriptorProto,
+ isLocal, defaultPayload);
- taskCommunicatorDescriptors = parsePlugin(taskCommunicators,
- (amPluginDescriptorProto == null ||
- amPluginDescriptorProto.getTaskCommunicatorsCount() == 0 ? null :
- amPluginDescriptorProto.getTaskCommunicatorsList()),
- tezYarnEnabled, uberEnabled);
LOG.info(buildPluginComponentLog(taskSchedulerDescriptors, taskSchedulers, "TaskSchedulers"));
@@ -494,12 +469,11 @@ public class DAGAppMaster extends AbstractService {
jobTokenSecretManager.addTokenForJob(
appAttemptID.getApplicationId().toString(), sessionToken);
- UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(amConf);
+
//service to handle requests to TaskUmbilicalProtocol
taskAttemptListener = createTaskAttemptListener(context,
- taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors,
- defaultPayload, isLocal);
+ taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors);
addIfService(taskAttemptListener, true);
containerSignatureMatcher = createContainerSignatureMatcher();
@@ -549,7 +523,7 @@ public class DAGAppMaster extends AbstractService {
this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService,
- taskSchedulerDescriptors, defaultPayload, isLocal);
+ taskSchedulerDescriptors, isLocal);
addIfService(taskSchedulerEventHandler, true);
if (enableWebUIService()) {
@@ -567,7 +541,7 @@ public class DAGAppMaster extends AbstractService {
taskSchedulerEventHandler);
addIfServiceDependency(taskSchedulerEventHandler, clientRpcServer);
- this.containerLauncherRouter = createContainerLauncherRouter(defaultPayload, containerLauncherDescriptors, isLocal);
+ this.containerLauncherRouter = createContainerLauncherRouter(containerLauncherDescriptors, isLocal);
addIfService(containerLauncherRouter, true);
dispatcher.register(NMCommunicatorEventType.class, containerLauncherRouter);
@@ -1077,12 +1051,9 @@ public class DAGAppMaster extends AbstractService {
protected TaskAttemptListener createTaskAttemptListener(AppContext context,
TaskHeartbeatHandler thh,
ContainerHeartbeatHandler chh,
- List<NamedEntityDescriptor> entityDescriptors,
- UserPayload defaultUserPayload,
- boolean isLocal) {
+ List<NamedEntityDescriptor> entityDescriptors) {
TaskAttemptListener lis =
- new TaskAttemptListenerImpTezDag(context, thh, chh,
- entityDescriptors, defaultUserPayload, isLocal);
+ new TaskAttemptListenerImpTezDag(context, thh, chh, entityDescriptors);
return lis;
}
@@ -1103,11 +1074,10 @@ public class DAGAppMaster extends AbstractService {
return chh;
}
- protected ContainerLauncherRouter createContainerLauncherRouter(UserPayload defaultPayload,
- List<NamedEntityDescriptor> containerLauncherDescriptors,
+ protected ContainerLauncherRouter createContainerLauncherRouter(List<NamedEntityDescriptor> containerLauncherDescriptors,
boolean isLocal) throws
UnknownHostException {
- return new ContainerLauncherRouter(defaultPayload, context, taskAttemptListener, workingDirectory,
+ return new ContainerLauncherRouter(context, taskAttemptListener, workingDirectory,
containerLauncherDescriptors, isLocal);
}
@@ -2401,41 +2371,106 @@ public class DAGAppMaster extends AbstractService {
TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT);
}
- private static List<NamedEntityDescriptor> parsePlugin(
- BiMap<String, Integer> pluginMap, List<TezNamedEntityDescriptorProto> namedEntityDescriptorProtos,
- boolean tezYarnEnabled, boolean uberEnabled) {
- int index = 0;
+ @VisibleForTesting
+ static void parseAllPlugins(
+ List<NamedEntityDescriptor> taskSchedulerDescriptors, BiMap<String, Integer> taskSchedulerPluginMap,
+ List<NamedEntityDescriptor> containerLauncherDescriptors, BiMap<String, Integer> containerLauncherPluginMap,
+ List<NamedEntityDescriptor> taskCommDescriptors, BiMap<String, Integer> taskCommPluginMap,
+ AMPluginDescriptorProto amPluginDescriptorProto, boolean isLocal, UserPayload defaultPayload) {
+
+ boolean tezYarnEnabled;
+ boolean uberEnabled;
+ if (!isLocal) {
+ if (amPluginDescriptorProto == null) {
+ tezYarnEnabled = true;
+ uberEnabled = false;
+ } else {
+ tezYarnEnabled = amPluginDescriptorProto.getContainersEnabled();
+ uberEnabled = amPluginDescriptorProto.getUberEnabled();
+ }
+ } else {
+ tezYarnEnabled = false;
+ uberEnabled = true;
+ }
+
+ parsePlugin(taskSchedulerDescriptors, taskSchedulerPluginMap,
+ (amPluginDescriptorProto == null || amPluginDescriptorProto.getTaskSchedulersCount() == 0 ?
+ null :
+ amPluginDescriptorProto.getTaskSchedulersList()),
+ tezYarnEnabled, uberEnabled, defaultPayload);
+ processSchedulerDescriptors(taskSchedulerDescriptors, isLocal, defaultPayload, taskSchedulerPluginMap);
- List<NamedEntityDescriptor> resultList = new LinkedList<>();
+ parsePlugin(containerLauncherDescriptors, containerLauncherPluginMap,
+ (amPluginDescriptorProto == null ||
+ amPluginDescriptorProto.getContainerLaunchersCount() == 0 ? null :
+ amPluginDescriptorProto.getContainerLaunchersList()),
+ tezYarnEnabled, uberEnabled, defaultPayload);
+
+ parsePlugin(taskCommDescriptors, taskCommPluginMap,
+ (amPluginDescriptorProto == null ||
+ amPluginDescriptorProto.getTaskCommunicatorsCount() == 0 ? null :
+ amPluginDescriptorProto.getTaskCommunicatorsList()),
+ tezYarnEnabled, uberEnabled, defaultPayload);
+ }
+
+
+ @VisibleForTesting
+ static void parsePlugin(List<NamedEntityDescriptor> resultList,
+ BiMap<String, Integer> pluginMap, List<TezNamedEntityDescriptorProto> namedEntityDescriptorProtos,
+ boolean tezYarnEnabled, boolean uberEnabled, UserPayload defaultPayload) {
if (tezYarnEnabled) {
// Default classnames will be populated by individual components
NamedEntityDescriptor r = new NamedEntityDescriptor(
- TezConstants.getTezYarnServicePluginName(), null);
- resultList.add(r);
- pluginMap.put(TezConstants.getTezYarnServicePluginName(), index);
- index++;
+ TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultPayload);
+ addDescriptor(resultList, pluginMap, r);
}
if (uberEnabled) {
// Default classnames will be populated by individual components
NamedEntityDescriptor r = new NamedEntityDescriptor(
- TezConstants.getTezUberServicePluginName(), null);
- resultList.add(r);
- pluginMap.put(TezConstants.getTezUberServicePluginName(), index);
- index++;
+ TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultPayload);
+ addDescriptor(resultList, pluginMap, r);
}
if (namedEntityDescriptorProtos != null) {
for (TezNamedEntityDescriptorProto namedEntityDescriptorProto : namedEntityDescriptorProtos) {
- resultList.add(DagTypeConverters
- .convertNamedDescriptorFromProto(namedEntityDescriptorProto));
- pluginMap.put(resultList.get(index).getEntityName(), index);
- index++;
+ NamedEntityDescriptor namedEntityDescriptor = DagTypeConverters
+ .convertNamedDescriptorFromProto(namedEntityDescriptorProto);
+ addDescriptor(resultList, pluginMap, namedEntityDescriptor);
+ }
+ }
+ }
+
+ @VisibleForTesting
+ static void addDescriptor(List<NamedEntityDescriptor> list, BiMap<String, Integer> pluginMap,
+ NamedEntityDescriptor namedEntityDescriptor) {
+ list.add(namedEntityDescriptor);
+ pluginMap.put(list.get(list.size() - 1).getEntityName(), list.size() - 1);
+ }
+
+ @VisibleForTesting
+ static void processSchedulerDescriptors(List<NamedEntityDescriptor> descriptors, boolean isLocal,
+ UserPayload defaultPayload,
+ BiMap<String, Integer> schedulerPluginMap) {
+ if (isLocal) {
+ Preconditions.checkState(descriptors.size() == 1 &&
+ descriptors.get(0).getEntityName().equals(TezConstants.getTezUberServicePluginName()));
+ } else {
+ boolean foundYarn = false;
+ for (int i = 0; i < descriptors.size(); i++) {
+ if (descriptors.get(i).getEntityName().equals(TezConstants.getTezYarnServicePluginName())) {
+ foundYarn = true;
+ }
+ }
+ if (!foundYarn) {
+ NamedEntityDescriptor yarnDescriptor =
+ new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(defaultPayload);
+ addDescriptor(descriptors, schedulerPluginMap, yarnDescriptor);
}
}
- return resultList;
}
String buildPluginComponentLog(List<NamedEntityDescriptor> namedEntityDescriptors, BiMap<String, Integer> map,
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 941e583..7b97738 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -27,7 +27,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
+import com.google.common.base.Preconditions;
import org.apache.commons.collections4.ListUtils;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConstants;
@@ -102,35 +102,19 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
public TaskAttemptListenerImpTezDag(AppContext context,
TaskHeartbeatHandler thh, ContainerHeartbeatHandler chh,
- List<NamedEntityDescriptor> taskCommunicatorDescriptors,
- UserPayload defaultUserPayload,
- boolean isPureLocalMode) {
+ List<NamedEntityDescriptor> taskCommunicatorDescriptors) {
super(TaskAttemptListenerImpTezDag.class.getName());
this.context = context;
this.taskHeartbeatHandler = thh;
this.containerHeartbeatHandler = chh;
- if (taskCommunicatorDescriptors == null || taskCommunicatorDescriptors.isEmpty()) {
- if (isPureLocalMode) {
- taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
- TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultUserPayload));
- } else {
- taskCommunicatorDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
- TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultUserPayload));
- }
- }
+ Preconditions.checkArgument(
+ taskCommunicatorDescriptors != null && !taskCommunicatorDescriptors.isEmpty(),
+ "TaskCommunicators must be specified");
this.taskCommunicators = new TaskCommunicator[taskCommunicatorDescriptors.size()];
this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorDescriptors.size()];
this.taskCommunicatorServiceWrappers = new ServicePluginLifecycleAbstractService[taskCommunicatorDescriptors.size()];
for (int i = 0 ; i < taskCommunicatorDescriptors.size() ; i++) {
- UserPayload userPayload;
- if (taskCommunicatorDescriptors.get(i).getEntityName()
- .equals(TezConstants.getTezYarnServicePluginName()) ||
- taskCommunicatorDescriptors.get(i).getEntityName()
- .equals(TezConstants.getTezUberServicePluginName())) {
- userPayload = defaultUserPayload;
- } else {
- userPayload = taskCommunicatorDescriptors.get(i).getUserPayload();
- }
+ UserPayload userPayload = taskCommunicatorDescriptors.get(i).getUserPayload();
taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, userPayload, i);
taskCommunicators[i] = createTaskCommunicator(taskCommunicatorDescriptors.get(i), i);
taskCommunicatorServiceWrappers[i] = new ServicePluginLifecycleAbstractService(taskCommunicators[i]);
@@ -154,36 +138,54 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
}
- private TaskCommunicator createTaskCommunicator(NamedEntityDescriptor taskCommDescriptor, int taskCommIndex) {
+ @VisibleForTesting
+ TaskCommunicator createTaskCommunicator(NamedEntityDescriptor taskCommDescriptor,
+ int taskCommIndex) {
if (taskCommDescriptor.getEntityName().equals(TezConstants.getTezYarnServicePluginName())) {
- LOG.info("Using Default Task Communicator");
- return createTezTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
- } else if (taskCommDescriptor.getEntityName().equals(TezConstants.getTezUberServicePluginName())) {
- LOG.info("Using Default Local Task Communicator");
- return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
+ return createDefaultTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
+ } else if (taskCommDescriptor.getEntityName()
+ .equals(TezConstants.getTezUberServicePluginName())) {
+ return createUberTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
} else {
- LOG.info("Using TaskCommunicator {}:{} " + taskCommDescriptor.getEntityName(), taskCommDescriptor.getClassName());
- Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
- .getClazz(taskCommDescriptor.getClassName());
- try {
- Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
- ctor.setAccessible(true);
- return ctor.newInstance(taskCommunicatorContexts[taskCommIndex]);
- } catch (NoSuchMethodException e) {
- throw new TezUncheckedException(e);
- } catch (InvocationTargetException e) {
- throw new TezUncheckedException(e);
- } catch (InstantiationException e) {
- throw new TezUncheckedException(e);
- } catch (IllegalAccessException e) {
- throw new TezUncheckedException(e);
- }
+ return createCustomTaskCommunicator(taskCommunicatorContexts[taskCommIndex],
+ taskCommDescriptor);
}
}
@VisibleForTesting
- protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
- return new TezTaskCommunicatorImpl(context);
+ TaskCommunicator createDefaultTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+ LOG.info("Using Default Task Communicator");
+ return new TezTaskCommunicatorImpl(taskCommunicatorContext);
+ }
+
+ @VisibleForTesting
+ TaskCommunicator createUberTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+ LOG.info("Using Default Local Task Communicator");
+ return new TezLocalTaskCommunicatorImpl(taskCommunicatorContext);
+ }
+
+ @VisibleForTesting
+ TaskCommunicator createCustomTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext,
+ NamedEntityDescriptor taskCommDescriptor) {
+ LOG.info("Using TaskCommunicator {}:{} " + taskCommDescriptor.getEntityName(),
+ taskCommDescriptor.getClassName());
+ Class<? extends TaskCommunicator> taskCommClazz =
+ (Class<? extends TaskCommunicator>) ReflectionUtils
+ .getClazz(taskCommDescriptor.getClassName());
+ try {
+ Constructor<? extends TaskCommunicator> ctor =
+ taskCommClazz.getConstructor(TaskCommunicatorContext.class);
+ ctor.setAccessible(true);
+ return ctor.newInstance(taskCommunicatorContext);
+ } catch (NoSuchMethodException e) {
+ throw new TezUncheckedException(e);
+ } catch (InvocationTargetException e) {
+ throw new TezUncheckedException(e);
+ } catch (InstantiationException e) {
+ throw new TezUncheckedException(e);
+ } catch (IllegalAccessException e) {
+ throw new TezUncheckedException(e);
+ }
}
public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index 2e8f218..3cc439f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -233,9 +233,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
private final boolean isSpeculationEnabled;
- private final int taskSchedulerIdentifier;
- private final int containerLauncherIdentifier;
- private final int taskCommunicatorIdentifier;
+ @VisibleForTesting
+ final int taskSchedulerIdentifier;
+ @VisibleForTesting
+ final int containerLauncherIdentifier;
+ @VisibleForTesting
+ final int taskCommunicatorIdentifier;
//fields initialized in init
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
index 2d56bfe..57b4aee 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -20,7 +20,7 @@ import java.net.UnknownHostException;
import java.util.List;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
+import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -48,8 +48,10 @@ public class ContainerLauncherRouter extends AbstractService
static final Logger LOG = LoggerFactory.getLogger(ContainerLauncherImpl.class);
- private final ContainerLauncher containerLaunchers[];
- private final ContainerLauncherContext containerLauncherContexts[];
+ @VisibleForTesting
+ final ContainerLauncher containerLaunchers[];
+ @VisibleForTesting
+ final ContainerLauncherContext containerLauncherContexts[];
protected final ServicePluginLifecycleAbstractService[] containerLauncherServiceWrappers;
private final AppContext appContext;
@@ -64,7 +66,7 @@ public class ContainerLauncherRouter extends AbstractService
}
// Accepting conf to setup final parameters, if required.
- public ContainerLauncherRouter(UserPayload defaultUserPayload, AppContext context,
+ public ContainerLauncherRouter(AppContext context,
TaskAttemptListener taskAttemptListener,
String workingDirectory,
List<NamedEntityDescriptor> containerLauncherDescriptors,
@@ -72,79 +74,91 @@ public class ContainerLauncherRouter extends AbstractService
super(ContainerLauncherRouter.class.getName());
this.appContext = context;
- if (containerLauncherDescriptors == null || containerLauncherDescriptors.isEmpty()) {
- if (isPureLocalMode) {
- containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
- TezConstants.getTezUberServicePluginName(), null).setUserPayload(defaultUserPayload));
- } else {
- containerLauncherDescriptors = Lists.newArrayList(new NamedEntityDescriptor(
- TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultUserPayload));
- }
- }
+ Preconditions.checkArgument(
+ containerLauncherDescriptors != null && !containerLauncherDescriptors.isEmpty(),
+ "ContainerLauncherDescriptors must be specified");
containerLauncherContexts = new ContainerLauncherContext[containerLauncherDescriptors.size()];
containerLaunchers = new ContainerLauncher[containerLauncherDescriptors.size()];
containerLauncherServiceWrappers = new ServicePluginLifecycleAbstractService[containerLauncherDescriptors.size()];
for (int i = 0; i < containerLauncherDescriptors.size(); i++) {
- UserPayload userPayload;
- if (containerLauncherDescriptors.get(i).getEntityName()
- .equals(TezConstants.getTezYarnServicePluginName()) ||
- containerLauncherDescriptors.get(i).getEntityName()
- .equals(TezConstants.getTezUberServicePluginName())) {
- userPayload = defaultUserPayload;
- } else {
- userPayload = containerLauncherDescriptors.get(i).getUserPayload();
- }
+ UserPayload userPayload = containerLauncherDescriptors.get(i).getUserPayload();
ContainerLauncherContext containerLauncherContext =
new ContainerLauncherContextImpl(context, taskAttemptListener, userPayload);
containerLauncherContexts[i] = containerLauncherContext;
containerLaunchers[i] = createContainerLauncher(containerLauncherDescriptors.get(i), context,
- containerLauncherContext, taskAttemptListener, workingDirectory, isPureLocalMode);
+ containerLauncherContext, taskAttemptListener, workingDirectory, i, isPureLocalMode);
containerLauncherServiceWrappers[i] = new ServicePluginLifecycleAbstractService(containerLaunchers[i]);
}
}
- private ContainerLauncher createContainerLauncher(NamedEntityDescriptor containerLauncherDescriptor,
- AppContext context,
- ContainerLauncherContext containerLauncherContext,
- TaskAttemptListener taskAttemptListener,
- String workingDirectory,
- boolean isPureLocalMode) throws
+ @VisibleForTesting
+ ContainerLauncher createContainerLauncher(
+ NamedEntityDescriptor containerLauncherDescriptor,
+ AppContext context,
+ ContainerLauncherContext containerLauncherContext,
+ TaskAttemptListener taskAttemptListener,
+ String workingDirectory,
+ int containerLauncherIndex,
+ boolean isPureLocalMode) throws
UnknownHostException {
if (containerLauncherDescriptor.getEntityName().equals(
TezConstants.getTezYarnServicePluginName())) {
- LOG.info("Creating DefaultContainerLauncher");
- return new ContainerLauncherImpl(containerLauncherContext);
+ return createYarnContainerLauncher(containerLauncherContext);
} else if (containerLauncherDescriptor.getEntityName()
.equals(TezConstants.getTezUberServicePluginName())) {
- LOG.info("Creating LocalContainerLauncher");
- // TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
- // extensive internals which are only available at runtime. Will likely require
- // some kind of runtime binding of parameters in the payload to work correctly.
- return
- new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener, workingDirectory, isPureLocalMode);
+ return createUberContainerLauncher(containerLauncherContext, context, taskAttemptListener,
+ workingDirectory, isPureLocalMode);
} else {
- LOG.info("Creating container launcher {}:{} ", containerLauncherDescriptor.getEntityName(), containerLauncherDescriptor.getClassName());
- Class<? extends ContainerLauncher> containerLauncherClazz =
- (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
- containerLauncherDescriptor.getClassName());
- try {
- Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
- .getConstructor(ContainerLauncherContext.class);
- ctor.setAccessible(true);
- return ctor.newInstance(containerLauncherContext);
- } catch (NoSuchMethodException e) {
- throw new TezUncheckedException(e);
- } catch (InvocationTargetException e) {
- throw new TezUncheckedException(e);
- } catch (InstantiationException e) {
- throw new TezUncheckedException(e);
- } catch (IllegalAccessException e) {
- throw new TezUncheckedException(e);
- }
+ return createCustomContainerLauncher(containerLauncherContext, containerLauncherDescriptor);
+ }
+ }
+
+ @VisibleForTesting
+ ContainerLauncher createYarnContainerLauncher(ContainerLauncherContext containerLauncherContext) {
+ LOG.info("Creating DefaultContainerLauncher");
+ return new ContainerLauncherImpl(containerLauncherContext);
+ }
+
+ @VisibleForTesting
+ ContainerLauncher createUberContainerLauncher(ContainerLauncherContext containerLauncherContext,
+ AppContext context,
+ TaskAttemptListener taskAttemptListener,
+ String workingDirectory,
+ boolean isPureLocalMode) throws
+ UnknownHostException {
+ LOG.info("Creating LocalContainerLauncher");
+ // TODO Post TEZ-2003. LocalContainerLauncher is special cased, since it makes use of
+ // extensive internals which are only available at runtime. Will likely require
+ // some kind of runtime binding of parameters in the payload to work correctly.
+ return
+ new LocalContainerLauncher(containerLauncherContext, context, taskAttemptListener,
+ workingDirectory, isPureLocalMode);
+ }
+
+ @VisibleForTesting
+ ContainerLauncher createCustomContainerLauncher(ContainerLauncherContext containerLauncherContext,
+ NamedEntityDescriptor containerLauncherDescriptor) {
+ LOG.info("Creating container launcher {}:{} ", containerLauncherDescriptor.getEntityName(),
+ containerLauncherDescriptor.getClassName());
+ Class<? extends ContainerLauncher> containerLauncherClazz =
+ (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
+ containerLauncherDescriptor.getClassName());
+ try {
+ Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
+ .getConstructor(ContainerLauncherContext.class);
+ ctor.setAccessible(true);
+ return ctor.newInstance(containerLauncherContext);
+ } catch (NoSuchMethodException e) {
+ throw new TezUncheckedException(e);
+ } catch (InvocationTargetException e) {
+ throw new TezUncheckedException(e);
+ } catch (InstantiationException e) {
+ throw new TezUncheckedException(e);
+ } catch (IllegalAccessException e) {
+ throw new TezUncheckedException(e);
}
- // TODO TEZ-2118 Handle routing to multiple launchers
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index c86f638..7c36232 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -22,7 +22,6 @@ import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
-import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
@@ -34,10 +33,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConstants;
-import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService;
import org.apache.tez.serviceplugins.api.TaskScheduler;
import org.apache.tez.serviceplugins.api.TaskSchedulerContext;
@@ -126,9 +123,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
private final boolean isPureLocalMode;
// If running in non local-only mode, the YARN task scheduler will always run to take care of
// registration with YARN and heartbeats to YARN.
- // Splitting registration and heartbeats is not straigh-forward due to the taskScheduler being
+ // Splitting registration and heartbeats is not straight-forward due to the taskScheduler being
// tied to a ContainerRequestType.
- private final int yarnTaskSchedulerIndex;
// Custom AppIds to avoid container conflicts if there's multiple sources
private final long SCHEDULER_APP_ID_BASE = 111101111;
private final long SCHEDULER_APP_ID_INCREMENT = 111111111;
@@ -153,9 +149,10 @@ public class TaskSchedulerEventHandler extends AbstractService implements
public TaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI,
- List<NamedEntityDescriptor> schedulerDescriptors, UserPayload defaultPayload,
- boolean isPureLocalMode) {
+ List<NamedEntityDescriptor> schedulerDescriptors, boolean isPureLocalMode) {
super(TaskSchedulerEventHandler.class.getName());
+ Preconditions.checkArgument(schedulerDescriptors != null && !schedulerDescriptors.isEmpty(),
+ "TaskSchedulerDescriptors must be specified");
this.appContext = appContext;
this.eventHandler = eventHandler;
this.clientService = clientService;
@@ -168,50 +165,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
this.webUI.setHistoryUrl(this.historyUrl);
}
- // Override everything for pure local mode
- if (isPureLocalMode) {
- this.taskSchedulerDescriptors = new NamedEntityDescriptor[]{
- new NamedEntityDescriptor(TezConstants.getTezUberServicePluginName(), null)
- .setUserPayload(defaultPayload)};
- this.yarnTaskSchedulerIndex = -1;
- } else {
- if (schedulerDescriptors == null || schedulerDescriptors.isEmpty()) {
- this.taskSchedulerDescriptors = new NamedEntityDescriptor[]{
- new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
- .setUserPayload(defaultPayload)};
- this.yarnTaskSchedulerIndex = 0;
- } else {
- // Ensure the YarnScheduler will be setup and note it's index. This will be responsible for heartbeats and YARN registration.
- int foundYarnTaskSchedulerIndex = -1;
-
- List<NamedEntityDescriptor> schedulerDescriptorList = new LinkedList<>();
- for (int i = 0 ; i < schedulerDescriptors.size() ; i++) {
- if (schedulerDescriptors.get(i).getEntityName().equals(
- TezConstants.getTezYarnServicePluginName())) {
- schedulerDescriptorList.add(
- new NamedEntityDescriptor(schedulerDescriptors.get(i).getEntityName(), null)
- .setUserPayload(
- defaultPayload));
- foundYarnTaskSchedulerIndex = i;
- } else if (schedulerDescriptors.get(i).getEntityName().equals(
- TezConstants.getTezUberServicePluginName())) {
- schedulerDescriptorList.add(
- new NamedEntityDescriptor(schedulerDescriptors.get(i).getEntityName(), null)
- .setUserPayload(
- defaultPayload));
- } else {
- schedulerDescriptorList.add(schedulerDescriptors.get(i));
- }
- }
- if (foundYarnTaskSchedulerIndex == -1) {
- schedulerDescriptorList.add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null).setUserPayload(
- defaultPayload));
- foundYarnTaskSchedulerIndex = schedulerDescriptorList.size() -1;
- }
- this.taskSchedulerDescriptors = schedulerDescriptorList.toArray(new NamedEntityDescriptor[schedulerDescriptorList.size()]);
- this.yarnTaskSchedulerIndex = foundYarnTaskSchedulerIndex;
- }
- }
+ this.taskSchedulerDescriptors = schedulerDescriptors.toArray(new NamedEntityDescriptor[schedulerDescriptors.size()]);
+
taskSchedulers = new TaskScheduler[this.taskSchedulerDescriptors.length];
taskSchedulerServiceWrappers = new ServicePluginLifecycleAbstractService[this.taskSchedulerDescriptors.length];
}
@@ -239,7 +194,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
private ExecutorService createAppCallbackExecutorService() {
return Executors.newSingleThreadExecutor(
- new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d").setDaemon(true)
+ new ThreadFactoryBuilder().setNameFormat("TaskSchedulerAppCallbackExecutor #%d")
+ .setDaemon(true)
.build());
}
@@ -428,7 +384,8 @@ public class TaskSchedulerEventHandler extends AbstractService implements
event);
}
- private TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
+ @VisibleForTesting
+ TaskScheduler createTaskScheduler(String host, int port, String trackingUrl,
AppContext appContext,
NamedEntityDescriptor taskSchedulerDescriptor,
long customAppIdIdentifier,
@@ -436,32 +393,57 @@ public class TaskSchedulerEventHandler extends AbstractService implements
TaskSchedulerContext rawContext =
new TaskSchedulerContextImpl(this, appContext, schedulerId, trackingUrl,
customAppIdIdentifier, host, port, taskSchedulerDescriptor.getUserPayload());
- TaskSchedulerContext wrappedContext = new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
+ TaskSchedulerContext wrappedContext = wrapTaskSchedulerContext(rawContext);
String schedulerName = taskSchedulerDescriptor.getEntityName();
if (schedulerName.equals(TezConstants.getTezYarnServicePluginName())) {
- LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
- return new YarnTaskSchedulerService(wrappedContext);
+ return createYarnTaskScheduler(wrappedContext, schedulerId);
} else if (schedulerName.equals(TezConstants.getTezUberServicePluginName())) {
- LOG.info("Creating TaskScheduler: Local TaskScheduler");
- return new LocalTaskSchedulerService(wrappedContext);
+ return createUberTaskScheduler(wrappedContext, schedulerId);
} else {
- LOG.info("Creating custom TaskScheduler {}:{}", taskSchedulerDescriptor.getEntityName(), taskSchedulerDescriptor.getClassName());
- Class<? extends TaskScheduler> taskSchedulerClazz =
- (Class<? extends TaskScheduler>) ReflectionUtils.getClazz(taskSchedulerDescriptor.getClassName());
- try {
- Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
- .getConstructor(TaskSchedulerContext.class);
- ctor.setAccessible(true);
- return ctor.newInstance(wrappedContext);
- } catch (NoSuchMethodException e) {
- throw new TezUncheckedException(e);
- } catch (InvocationTargetException e) {
- throw new TezUncheckedException(e);
- } catch (InstantiationException e) {
- throw new TezUncheckedException(e);
- } catch (IllegalAccessException e) {
- throw new TezUncheckedException(e);
- }
+ return createCustomTaskScheduler(wrappedContext, taskSchedulerDescriptor, schedulerId);
+ }
+ }
+
+ @VisibleForTesting
+ TaskSchedulerContext wrapTaskSchedulerContext(TaskSchedulerContext rawContext) {
+ return new TaskSchedulerContextImplWrapper(rawContext, appCallbackExecutor);
+ }
+
+ @VisibleForTesting
+ TaskScheduler createYarnTaskScheduler(TaskSchedulerContext taskSchedulerContext,
+ int schedulerId) {
+ LOG.info("Creating TaskScheduler: YarnTaskSchedulerService");
+ return new YarnTaskSchedulerService(taskSchedulerContext);
+ }
+
+ @VisibleForTesting
+ TaskScheduler createUberTaskScheduler(TaskSchedulerContext taskSchedulerContext,
+ int schedulerId) {
+ LOG.info("Creating TaskScheduler: Local TaskScheduler");
+ return new LocalTaskSchedulerService(taskSchedulerContext);
+ }
+
+ TaskScheduler createCustomTaskScheduler(TaskSchedulerContext taskSchedulerContext,
+ NamedEntityDescriptor taskSchedulerDescriptor,
+ int schedulerId) {
+ LOG.info("Creating custom TaskScheduler {}:{}", taskSchedulerDescriptor.getEntityName(),
+ taskSchedulerDescriptor.getClassName());
+ Class<? extends TaskScheduler> taskSchedulerClazz =
+ (Class<? extends TaskScheduler>) ReflectionUtils
+ .getClazz(taskSchedulerDescriptor.getClassName());
+ try {
+ Constructor<? extends TaskScheduler> ctor = taskSchedulerClazz
+ .getConstructor(TaskSchedulerContext.class);
+ ctor.setAccessible(true);
+ return ctor.newInstance(taskSchedulerContext);
+ } catch (NoSuchMethodException e) {
+ throw new TezUncheckedException(e);
+ } catch (InvocationTargetException e) {
+ throw new TezUncheckedException(e);
+ } catch (InstantiationException e) {
+ throw new TezUncheckedException(e);
+ } catch (IllegalAccessException e) {
+ throw new TezUncheckedException(e);
}
}
@@ -797,9 +779,4 @@ public class TaskSchedulerEventHandler extends AbstractService implements
return historyUrl;
}
- @VisibleForTesting
- @InterfaceAudience.Private
- ExecutorService getContextExecutorService() {
- return appCallbackExecutor;
- }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index 0723dbc..2e6e568 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -511,8 +511,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
// use mock container launcher for tests
@Override
- protected ContainerLauncherRouter createContainerLauncherRouter(final UserPayload defaultUserPayload,
- List<NamedEntityDescriptor> containerLauncherDescirptors,
+ protected ContainerLauncherRouter createContainerLauncherRouter(List<NamedEntityDescriptor> containerLauncherDescirptors,
boolean isLocal)
throws UnknownHostException {
return new ContainerLauncherRouter(containerLauncher, getContext());
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
new file mode 100644
index 0000000..fa5d87c
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
@@ -0,0 +1,300 @@
+/*
+ * 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.tez.dag.app;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezUserPayloadProto;
+import org.junit.Test;
+
+public class TestDAGAppMaster {
+
+ private static final String TEST_KEY = "TEST_KEY";
+ private static final String TEST_VAL = "TEST_VAL";
+ private static final String TS_NAME = "TS";
+ private static final String CL_NAME = "CL";
+ private static final String TC_NAME = "TC";
+ private static final String CLASS_SUFFIX = "_CLASS";
+
+ @Test(timeout = 5000)
+ public void testPluginParsing() throws IOException {
+ BiMap<String, Integer> pluginMap = HashBiMap.create();
+ Configuration conf = new Configuration(false);
+ conf.set("testkey", "testval");
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+ List<TezNamedEntityDescriptorProto> entityDescriptors = new LinkedList<>();
+ List<NamedEntityDescriptor> entities;
+
+ // Test empty descriptor list, yarn enabled
+ pluginMap.clear();
+ entities = new LinkedList<>();
+ DAGAppMaster.parsePlugin(entities, pluginMap, null, true, false, defaultPayload);
+ assertEquals(1, pluginMap.size());
+ assertEquals(1, entities.size());
+ assertTrue(pluginMap.containsKey(TezConstants.getTezYarnServicePluginName()));
+ assertTrue(0 == pluginMap.get(TezConstants.getTezYarnServicePluginName()));
+ assertEquals("testval",
+ TezUtils.createConfFromUserPayload(entities.get(0).getUserPayload()).get("testkey"));
+
+ // Test empty descriptor list, uber enabled
+ pluginMap.clear();
+ entities = new LinkedList<>();
+ DAGAppMaster.parsePlugin(entities, pluginMap, null, false, true, defaultPayload);
+ assertEquals(1, pluginMap.size());
+ assertEquals(1, entities.size());
+ assertTrue(pluginMap.containsKey(TezConstants.getTezUberServicePluginName()));
+ assertTrue(0 == pluginMap.get(TezConstants.getTezUberServicePluginName()));
+ assertEquals("testval",
+ TezUtils.createConfFromUserPayload(entities.get(0).getUserPayload()).get("testkey"));
+
+ // Test empty descriptor list, yarn enabled, uber enabled
+ pluginMap.clear();
+ entities = new LinkedList<>();
+ DAGAppMaster.parsePlugin(entities, pluginMap, null, true, true, defaultPayload);
+ assertEquals(2, pluginMap.size());
+ assertEquals(2, entities.size());
+ assertTrue(pluginMap.containsKey(TezConstants.getTezYarnServicePluginName()));
+ assertTrue(0 == pluginMap.get(TezConstants.getTezYarnServicePluginName()));
+ assertTrue(pluginMap.containsKey(TezConstants.getTezUberServicePluginName()));
+ assertTrue(1 == pluginMap.get(TezConstants.getTezUberServicePluginName()));
+
+
+ String pluginName = "d1";
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ TezNamedEntityDescriptorProto d1 =
+ TezNamedEntityDescriptorProto.newBuilder().setName(pluginName).setEntityDescriptor(
+ DAGProtos.TezEntityDescriptorProto.newBuilder().setClassName("d1Class")
+ .setTezUserPayload(
+ TezUserPayloadProto.newBuilder()
+ .setUserPayload(ByteString.copyFrom(bb)))).build();
+ entityDescriptors.add(d1);
+
+ // Test descriptor, no yarn, no uber
+ pluginMap.clear();
+ entities = new LinkedList<>();
+ DAGAppMaster.parsePlugin(entities, pluginMap, entityDescriptors, false, false, defaultPayload);
+ assertEquals(1, pluginMap.size());
+ assertEquals(1, entities.size());
+ assertTrue(pluginMap.containsKey(pluginName));
+ assertTrue(0 == pluginMap.get(pluginName));
+
+ // Test descriptor, yarn and uber
+ pluginMap.clear();
+ entities = new LinkedList<>();
+ DAGAppMaster.parsePlugin(entities, pluginMap, entityDescriptors, true, true, defaultPayload);
+ assertEquals(3, pluginMap.size());
+ assertEquals(3, entities.size());
+ assertTrue(pluginMap.containsKey(TezConstants.getTezYarnServicePluginName()));
+ assertTrue(0 == pluginMap.get(TezConstants.getTezYarnServicePluginName()));
+ assertTrue(pluginMap.containsKey(TezConstants.getTezUberServicePluginName()));
+ assertTrue(1 == pluginMap.get(TezConstants.getTezUberServicePluginName()));
+ assertTrue(pluginMap.containsKey(pluginName));
+ assertTrue(2 == pluginMap.get(pluginName));
+ entityDescriptors.clear();
+ }
+
+
+ @Test(timeout = 5000)
+ public void testParseAllPluginsNoneSpecified() throws IOException {
+ Configuration conf = new Configuration(false);
+ conf.set(TEST_KEY, TEST_VAL);
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+ List<NamedEntityDescriptor> tsDescriptors;
+ BiMap<String, Integer> tsMap;
+ List<NamedEntityDescriptor> clDescriptors;
+ BiMap<String, Integer> clMap;
+ List<NamedEntityDescriptor> tcDescriptors;
+ BiMap<String, Integer> tcMap;
+
+
+ // No plugins. Non local
+ tsDescriptors = Lists.newLinkedList();
+ tsMap = HashBiMap.create();
+ clDescriptors = Lists.newLinkedList();
+ clMap = HashBiMap.create();
+ tcDescriptors = Lists.newLinkedList();
+ tcMap = HashBiMap.create();
+ DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, clDescriptors, clMap, tcDescriptors, tcMap,
+ null, false, defaultPayload);
+ verifyDescAndMap(tsDescriptors, tsMap, 1, true, TezConstants.getTezYarnServicePluginName());
+ verifyDescAndMap(clDescriptors, clMap, 1, true, TezConstants.getTezYarnServicePluginName());
+ verifyDescAndMap(tcDescriptors, tcMap, 1, true, TezConstants.getTezYarnServicePluginName());
+
+ // No plugins. Local
+ tsDescriptors = Lists.newLinkedList();
+ tsMap = HashBiMap.create();
+ clDescriptors = Lists.newLinkedList();
+ clMap = HashBiMap.create();
+ tcDescriptors = Lists.newLinkedList();
+ tcMap = HashBiMap.create();
+ DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, clDescriptors, clMap, tcDescriptors, tcMap,
+ null, true, defaultPayload);
+ verifyDescAndMap(tsDescriptors, tsMap, 1, true, TezConstants.getTezUberServicePluginName());
+ verifyDescAndMap(clDescriptors, clMap, 1, true, TezConstants.getTezUberServicePluginName());
+ verifyDescAndMap(tcDescriptors, tcMap, 1, true, TezConstants.getTezUberServicePluginName());
+ }
+
+ @Test(timeout = 5000)
+ public void testParseAllPluginsOnlyCustomSpecified() throws IOException {
+ Configuration conf = new Configuration(false);
+ conf.set(TEST_KEY, TEST_VAL);
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+ TezUserPayloadProto payloadProto = TezUserPayloadProto.newBuilder()
+ .setUserPayload(ByteString.copyFrom(defaultPayload.getPayload())).build();
+
+ AMPluginDescriptorProto proto = createAmPluginDescriptor(false, false, true, payloadProto);
+
+ List<NamedEntityDescriptor> tsDescriptors;
+ BiMap<String, Integer> tsMap;
+ List<NamedEntityDescriptor> clDescriptors;
+ BiMap<String, Integer> clMap;
+ List<NamedEntityDescriptor> tcDescriptors;
+ BiMap<String, Integer> tcMap;
+
+
+ // Only plugin, Yarn.
+ tsDescriptors = Lists.newLinkedList();
+ tsMap = HashBiMap.create();
+ clDescriptors = Lists.newLinkedList();
+ clMap = HashBiMap.create();
+ tcDescriptors = Lists.newLinkedList();
+ tcMap = HashBiMap.create();
+ DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, clDescriptors, clMap, tcDescriptors, tcMap,
+ proto, false, defaultPayload);
+ verifyDescAndMap(tsDescriptors, tsMap, 2, true, TS_NAME,
+ TezConstants.getTezYarnServicePluginName());
+ verifyDescAndMap(clDescriptors, clMap, 1, true, CL_NAME);
+ verifyDescAndMap(tcDescriptors, tcMap, 1, true, TC_NAME);
+ assertEquals(TS_NAME + CLASS_SUFFIX, tsDescriptors.get(0).getClassName());
+ assertEquals(CL_NAME + CLASS_SUFFIX, clDescriptors.get(0).getClassName());
+ assertEquals(TC_NAME + CLASS_SUFFIX, tcDescriptors.get(0).getClassName());
+ }
+
+ @Test(timeout = 5000)
+ public void testParseAllPluginsCustomAndYarnSpecified() throws IOException {
+ Configuration conf = new Configuration(false);
+ conf.set(TEST_KEY, TEST_VAL);
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+ TezUserPayloadProto payloadProto = TezUserPayloadProto.newBuilder()
+ .setUserPayload(ByteString.copyFrom(defaultPayload.getPayload())).build();
+
+ AMPluginDescriptorProto proto = createAmPluginDescriptor(true, false, true, payloadProto);
+
+ List<NamedEntityDescriptor> tsDescriptors;
+ BiMap<String, Integer> tsMap;
+ List<NamedEntityDescriptor> clDescriptors;
+ BiMap<String, Integer> clMap;
+ List<NamedEntityDescriptor> tcDescriptors;
+ BiMap<String, Integer> tcMap;
+
+
+ // Only plugin, Yarn.
+ tsDescriptors = Lists.newLinkedList();
+ tsMap = HashBiMap.create();
+ clDescriptors = Lists.newLinkedList();
+ clMap = HashBiMap.create();
+ tcDescriptors = Lists.newLinkedList();
+ tcMap = HashBiMap.create();
+ DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, clDescriptors, clMap, tcDescriptors, tcMap,
+ proto, false, defaultPayload);
+ verifyDescAndMap(tsDescriptors, tsMap, 2, true, TezConstants.getTezYarnServicePluginName(),
+ TS_NAME);
+ verifyDescAndMap(clDescriptors, clMap, 2, true, TezConstants.getTezYarnServicePluginName(),
+ CL_NAME);
+ verifyDescAndMap(tcDescriptors, tcMap, 2, true, TezConstants.getTezYarnServicePluginName(),
+ TC_NAME);
+ assertNull(tsDescriptors.get(0).getClassName());
+ assertNull(clDescriptors.get(0).getClassName());
+ assertNull(tcDescriptors.get(0).getClassName());
+ assertEquals(TS_NAME + CLASS_SUFFIX, tsDescriptors.get(1).getClassName());
+ assertEquals(CL_NAME + CLASS_SUFFIX, clDescriptors.get(1).getClassName());
+ assertEquals(TC_NAME + CLASS_SUFFIX, tcDescriptors.get(1).getClassName());
+ }
+
+ private void verifyDescAndMap(List<NamedEntityDescriptor> descriptors, BiMap<String, Integer> map,
+ int numExpected, boolean verifyPayload,
+ String... expectedNames) throws
+ IOException {
+ Preconditions.checkArgument(expectedNames.length == numExpected);
+ assertEquals(numExpected, descriptors.size());
+ assertEquals(numExpected, map.size());
+ for (int i = 0; i < numExpected; i++) {
+ assertEquals(expectedNames[i], descriptors.get(i).getEntityName());
+ if (verifyPayload) {
+ assertEquals(TEST_VAL,
+ TezUtils.createConfFromUserPayload(descriptors.get(0).getUserPayload()).get(TEST_KEY));
+ }
+ assertTrue(map.get(expectedNames[i]) == i);
+ assertTrue(map.inverse().get(i) == expectedNames[i]);
+ }
+ }
+
+ private AMPluginDescriptorProto createAmPluginDescriptor(boolean enableYarn, boolean enableUber,
+ boolean addCustom,
+ TezUserPayloadProto payloadProto) {
+ AMPluginDescriptorProto.Builder builder = AMPluginDescriptorProto.newBuilder()
+ .setUberEnabled(enableUber)
+ .setContainersEnabled(enableYarn);
+ if (addCustom) {
+ builder.addTaskSchedulers(
+ TezNamedEntityDescriptorProto.newBuilder()
+ .setName(TS_NAME)
+ .setEntityDescriptor(
+ DAGProtos.TezEntityDescriptorProto.newBuilder()
+ .setClassName(TS_NAME + CLASS_SUFFIX)
+ .setTezUserPayload(payloadProto)))
+ .addContainerLaunchers(
+ TezNamedEntityDescriptorProto.newBuilder()
+ .setName(CL_NAME)
+ .setEntityDescriptor(
+ DAGProtos.TezEntityDescriptorProto.newBuilder()
+ .setClassName(CL_NAME + CLASS_SUFFIX)
+ .setTezUserPayload(payloadProto)))
+ .addTaskCommunicators(
+ TezNamedEntityDescriptorProto.newBuilder()
+ .setName(TC_NAME)
+ .setEntityDescriptor(
+ DAGProtos.TezEntityDescriptorProto.newBuilder()
+ .setClassName(TC_NAME + CLASS_SUFFIX)
+ .setTezUserPayload(payloadProto)));
+ }
+ return builder.build();
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
index 1cb69a8..639c487 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -34,6 +34,7 @@ import java.util.List;
import java.util.Map;
import java.util.Random;
+import com.google.common.collect.Lists;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
@@ -52,7 +53,9 @@ import org.apache.tez.common.security.JobTokenIdentifier;
import org.apache.tez.common.security.JobTokenSecretManager;
import org.apache.tez.common.security.TokenCache;
import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TaskCommunicator;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
@@ -146,7 +149,10 @@ public class TestTaskAttemptListenerImplTezDag {
throw new TezUncheckedException(e);
}
taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, defaultPayload, false);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class),
+ Lists.newArrayList(
+ new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(defaultPayload)));
TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
@@ -301,7 +307,7 @@ public class TestTaskAttemptListenerImplTezDag {
// TODO TEZ-2003 Move this into TestTezTaskCommunicator. Potentially other tests as well.
@Test (timeout= 5000)
- public void testPortRange_NotSpecified() {
+ public void testPortRange_NotSpecified() throws IOException {
Configuration conf = new Configuration();
JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(
"fakeIdentifier"));
@@ -309,14 +315,11 @@ public class TestTaskAttemptListenerImplTezDag {
new JobTokenSecretManager());
sessionToken.setService(identifier.getJobId());
TokenCache.setSessionToken(sessionToken, credentials);
- UserPayload userPayload = null;
- try {
- userPayload = TezUtils.createUserPayloadFromConf(conf);
- } catch (IOException e) {
- throw new TezUncheckedException(e);
- }
+ UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, userPayload, false);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), Lists.newArrayList(
+ new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(userPayload)));
// no exception happen, should started properly
taskAttemptListener.init(conf);
taskAttemptListener.start();
@@ -335,14 +338,12 @@ public class TestTaskAttemptListenerImplTezDag {
TokenCache.setSessionToken(sessionToken, credentials);
conf.set(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE, port + "-" + port);
- UserPayload userPayload = null;
- try {
- userPayload = TezUtils.createUserPayloadFromConf(conf);
- } catch (IOException e) {
- throw new TezUncheckedException(e);
- }
+ UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
+
taskAttemptListener = new TaskAttemptListenerImpTezDag(appContext,
- mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, userPayload, false);
+ mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), Lists
+ .newArrayList(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null)
+ .setUserPayload(userPayload)));
taskAttemptListener.init(conf);
taskAttemptListener.start();
int resultedPort = taskAttemptListener.getTaskCommunicator(0).getAddress().getPort();
@@ -398,16 +399,13 @@ public class TestTaskAttemptListenerImplTezDag {
public TaskAttemptListenerImplForTest(AppContext context,
TaskHeartbeatHandler thh,
ContainerHeartbeatHandler chh,
- List<NamedEntityDescriptor> taskCommDescriptors,
- UserPayload userPayload,
- boolean isPureLocalMode) {
- super(context, thh, chh, taskCommDescriptors, userPayload,
- isPureLocalMode);
+ List<NamedEntityDescriptor> taskCommDescriptors) {
+ super(context, thh, chh, taskCommDescriptors);
}
@Override
- protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
- return new TezTaskCommunicatorImplForTest(context);
+ TaskCommunicator createDefaultTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+ return new TezTaskCommunicatorImplForTest(taskCommunicatorContext);
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
index 1c82bd8..abb5e42 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
@@ -26,6 +26,7 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
+import com.google.common.collect.Lists;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@@ -37,7 +38,9 @@ import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
@@ -83,7 +86,8 @@ public class TestTaskAttemptListenerImplTezDag2 {
UserPayload userPayload = TezUtils.createUserPayloadFromConf(conf);
TaskAttemptListenerImpTezDag taskAttemptListener =
new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
- mock(ContainerHeartbeatHandler.class), null, userPayload, false);
+ mock(ContainerHeartbeatHandler.class), Lists.newArrayList(new NamedEntityDescriptor(
+ TezConstants.getTezYarnServicePluginName(), null).setUserPayload(userPayload)));
TaskSpec taskSpec1 = mock(TaskSpec.class);
TezTaskAttemptID taskAttemptId1 = mock(TezTaskAttemptID.class);
http://git-wip-us.apache.org/repos/asf/tez/blob/4a21040a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
new file mode 100644
index 0000000..c76aa50
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java
@@ -0,0 +1,369 @@
+/*
+ * 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.tez.dag.app;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.NamedEntityDescriptor;
+import org.apache.tez.dag.api.TaskCommunicator;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.serviceplugins.api.ContainerEndReason;
+import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestTaskCommunicatorManager {
+
+ @Before
+ @After
+ public void reset() {
+ TaskCommManagerForMultipleCommTest.reset();
+ }
+
+ @Test(timeout = 5000)
+ public void testNoTaskCommSpecified() throws IOException {
+
+ AppContext appContext = mock(AppContext.class);
+ TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class);
+ ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+
+ try {
+ new TaskCommManagerForMultipleCommTest(appContext, thh, chh, null);
+ fail("Initialization should have failed without a TaskComm specified");
+ } catch (IllegalArgumentException e) {
+
+ }
+
+
+ }
+
+ @Test(timeout = 5000)
+ public void testCustomTaskCommSpecified() throws IOException {
+
+ AppContext appContext = mock(AppContext.class);
+ TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class);
+ ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+
+ String customTaskCommName = "customTaskComm";
+ List<NamedEntityDescriptor> taskCommDescriptors = new LinkedList<>();
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ UserPayload customPayload = UserPayload.create(bb);
+ taskCommDescriptors.add(
+ new NamedEntityDescriptor(customTaskCommName, FakeTaskComm.class.getName())
+ .setUserPayload(customPayload));
+
+ TaskCommManagerForMultipleCommTest tcm =
+ new TaskCommManagerForMultipleCommTest(appContext, thh, chh, taskCommDescriptors);
+
+ try {
+ tcm.init(new Configuration(false));
+ tcm.start();
+
+ assertEquals(1, tcm.getNumTaskComms());
+ assertFalse(tcm.getYarnTaskCommCreated());
+ assertFalse(tcm.getUberTaskCommCreated());
+
+ assertEquals(customTaskCommName, tcm.getTaskCommName(0));
+ assertEquals(bb, tcm.getTaskCommContext(0).getInitialUserPayload().getPayload());
+
+ } finally {
+ tcm.stop();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testMultipleTaskComms() throws IOException {
+
+ AppContext appContext = mock(AppContext.class);
+ TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class);
+ ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+ Configuration conf = new Configuration(false);
+ conf.set("testkey", "testvalue");
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+ String customTaskCommName = "customTaskComm";
+ List<NamedEntityDescriptor> taskCommDescriptors = new LinkedList<>();
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ UserPayload customPayload = UserPayload.create(bb);
+ taskCommDescriptors.add(
+ new NamedEntityDescriptor(customTaskCommName, FakeTaskComm.class.getName())
+ .setUserPayload(customPayload));
+ taskCommDescriptors
+ .add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultPayload));
+
+ TaskCommManagerForMultipleCommTest tcm =
+ new TaskCommManagerForMultipleCommTest(appContext, thh, chh, taskCommDescriptors);
+
+ try {
+ tcm.init(new Configuration(false));
+ tcm.start();
+
+ assertEquals(2, tcm.getNumTaskComms());
+ assertTrue(tcm.getYarnTaskCommCreated());
+ assertFalse(tcm.getUberTaskCommCreated());
+
+ assertEquals(customTaskCommName, tcm.getTaskCommName(0));
+ assertEquals(bb, tcm.getTaskCommContext(0).getInitialUserPayload().getPayload());
+
+ assertEquals(TezConstants.getTezYarnServicePluginName(), tcm.getTaskCommName(1));
+ Configuration confParsed = TezUtils
+ .createConfFromUserPayload(tcm.getTaskCommContext(1).getInitialUserPayload());
+ assertEquals("testvalue", confParsed.get("testkey"));
+ } finally {
+ tcm.stop();
+ }
+ }
+
+ @Test(timeout = 5000)
+ public void testEventRouting() throws Exception {
+
+ AppContext appContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+ NodeId nodeId = NodeId.newInstance("host1", 3131);
+ when(appContext.getAllContainers().get(any(ContainerId.class)).getContainer().getNodeId())
+ .thenReturn(nodeId);
+ TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class);
+ ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+ Configuration conf = new Configuration(false);
+ conf.set("testkey", "testvalue");
+ UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(conf);
+
+ String customTaskCommName = "customTaskComm";
+ List<NamedEntityDescriptor> taskCommDescriptors = new LinkedList<>();
+ ByteBuffer bb = ByteBuffer.allocate(4);
+ bb.putInt(0, 3);
+ UserPayload customPayload = UserPayload.create(bb);
+ taskCommDescriptors.add(
+ new NamedEntityDescriptor(customTaskCommName, FakeTaskComm.class.getName())
+ .setUserPayload(customPayload));
+ taskCommDescriptors
+ .add(new NamedEntityDescriptor(TezConstants.getTezYarnServicePluginName(), null).setUserPayload(defaultPayload));
+
+ TaskCommManagerForMultipleCommTest tcm =
+ new TaskCommManagerForMultipleCommTest(appContext, thh, chh, taskCommDescriptors);
+
+ try {
+ tcm.init(new Configuration(false));
+ tcm.start();
+
+ assertEquals(2, tcm.getNumTaskComms());
+ assertTrue(tcm.getYarnTaskCommCreated());
+ assertFalse(tcm.getUberTaskCommCreated());
+
+ verify(tcm.getTestTaskComm(0)).initialize();
+ verify(tcm.getTestTaskComm(0)).start();
+ verify(tcm.getTestTaskComm(1)).initialize();
+ verify(tcm.getTestTaskComm(1)).start();
+
+
+ ContainerId containerId1 = mock(ContainerId.class);
+ tcm.registerRunningContainer(containerId1, 0);
+ verify(tcm.getTestTaskComm(0)).registerRunningContainer(eq(containerId1), eq("host1"),
+ eq(3131));
+
+ ContainerId containerId2 = mock(ContainerId.class);
+ tcm.registerRunningContainer(containerId2, 1);
+ verify(tcm.getTestTaskComm(1)).registerRunningContainer(eq(containerId2), eq("host1"),
+ eq(3131));
+
+ } finally {
+ tcm.stop();
+ verify(tcm.getTaskCommunicator(0)).shutdown();
+ verify(tcm.getTaskCommunicator(1)).shutdown();
+ }
+ }
+
+
+ static class TaskCommManagerForMultipleCommTest extends TaskAttemptListenerImpTezDag {
+
+ // All variables setup as static since methods being overridden are invoked by the ContainerLauncherRouter ctor,
+ // and regular variables will not be initialized at this point.
+ private static final AtomicInteger numTaskComms = new AtomicInteger(0);
+ private static final Set<Integer> taskCommIndices = new HashSet<>();
+ private static final TaskCommunicator yarnTaskComm = mock(TaskCommunicator.class);
+ private static final TaskCommunicator uberTaskComm = mock(TaskCommunicator.class);
+ private static final AtomicBoolean yarnTaskCommCreated = new AtomicBoolean(false);
+ private static final AtomicBoolean uberTaskCommCreated = new AtomicBoolean(false);
+
+ private static final List<TaskCommunicatorContext> taskCommContexts =
+ new LinkedList<>();
+ private static final List<String> taskCommNames = new LinkedList<>();
+ private static final List<TaskCommunicator> testTaskComms = new LinkedList<>();
+
+
+ public static void reset() {
+ numTaskComms.set(0);
+ taskCommIndices.clear();
+ yarnTaskCommCreated.set(false);
+ uberTaskCommCreated.set(false);
+ taskCommContexts.clear();
+ taskCommNames.clear();
+ testTaskComms.clear();
+ }
+
+ public TaskCommManagerForMultipleCommTest(AppContext context,
+ TaskHeartbeatHandler thh,
+ ContainerHeartbeatHandler chh,
+ List<NamedEntityDescriptor> taskCommunicatorDescriptors) {
+ super(context, thh, chh, taskCommunicatorDescriptors);
+ }
+
+ @Override
+ TaskCommunicator createTaskCommunicator(NamedEntityDescriptor taskCommDescriptor,
+ int taskCommIndex) {
+ numTaskComms.incrementAndGet();
+ boolean added = taskCommIndices.add(taskCommIndex);
+ assertTrue("Cannot add multiple taskComms with the same index", added);
+ taskCommNames.add(taskCommDescriptor.getEntityName());
+ return super.createTaskCommunicator(taskCommDescriptor, taskCommIndex);
+ }
+
+ @Override
+ TaskCommunicator createDefaultTaskCommunicator(
+ TaskCommunicatorContext taskCommunicatorContext) {
+ taskCommContexts.add(taskCommunicatorContext);
+ yarnTaskCommCreated.set(true);
+ testTaskComms.add(yarnTaskComm);
+ return yarnTaskComm;
+ }
+
+ @Override
+ TaskCommunicator createUberTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext) {
+ taskCommContexts.add(taskCommunicatorContext);
+ uberTaskCommCreated.set(true);
+ testTaskComms.add(uberTaskComm);
+ return uberTaskComm;
+ }
+
+ @Override
+ TaskCommunicator createCustomTaskCommunicator(TaskCommunicatorContext taskCommunicatorContext,
+ NamedEntityDescriptor taskCommDescriptor) {
+ taskCommContexts.add(taskCommunicatorContext);
+ TaskCommunicator spyComm =
+ spy(super.createCustomTaskCommunicator(taskCommunicatorContext, taskCommDescriptor));
+ testTaskComms.add(spyComm);
+ return spyComm;
+ }
+
+ public static int getNumTaskComms() {
+ return numTaskComms.get();
+ }
+
+ public static boolean getYarnTaskCommCreated() {
+ return yarnTaskCommCreated.get();
+ }
+
+ public static boolean getUberTaskCommCreated() {
+ return uberTaskCommCreated.get();
+ }
+
+ public static TaskCommunicatorContext getTaskCommContext(int taskCommIndex) {
+ return taskCommContexts.get(taskCommIndex);
+ }
+
+ public static String getTaskCommName(int taskCommIndex) {
+ return taskCommNames.get(taskCommIndex);
+ }
+
+ public static TaskCommunicator getTestTaskComm(int taskCommIndex) {
+ return testTaskComms.get(taskCommIndex);
+ }
+ }
+
+ public static class FakeTaskComm extends TaskCommunicator {
+
+ public FakeTaskComm(TaskCommunicatorContext taskCommunicatorContext) {
+ super(taskCommunicatorContext);
+ }
+
+ @Override
+ public void registerRunningContainer(ContainerId containerId, String hostname, int port) {
+
+ }
+
+ @Override
+ public void registerContainerEnd(ContainerId containerId, ContainerEndReason endReason) {
+
+ }
+
+ @Override
+ public void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
+ Map<String, LocalResource> additionalResources,
+ Credentials credentials, boolean credentialsChanged,
+ int priority) {
+
+ }
+
+ @Override
+ public void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID,
+ TaskAttemptEndReason endReason) {
+
+ }
+
+ @Override
+ public InetSocketAddress getAddress() {
+ return null;
+ }
+
+ @Override
+ public void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception {
+
+ }
+
+ @Override
+ public void dagComplete(String dagName) {
+
+ }
+
+ @Override
+ public Object getMetaInfo() {
+ return null;
+ }
+ }
+}