/**
* 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.test;
import java.nio.ByteBuffer;
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.hadoop.yarn.conf.YarnConfiguration;
import org.apache.tez.client.TezClientUtils;
import org.apache.tez.client.TezClient;
import org.apache.tez.dag.api.DAG;
import org.apache.tez.dag.api.DataSinkDescriptor;
import org.apache.tez.dag.api.OutputCommitterDescriptor;
import org.apache.tez.dag.api.OutputDescriptor;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.client.DAGClient;
import org.apache.tez.dag.api.client.DAGStatus;
import org.apache.tez.dag.api.client.DAGStatus.State;
import org.apache.tez.test.dag.MultiAttemptDAG;
import org.apache.tez.test.dag.SimpleVTestDAG;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.IOException;
import java.util.Random;
public class TestDAGRecovery2 {
private static final Log LOG = LogFactory.getLog(TestDAGRecovery2.class);
private static Configuration conf = new Configuration();
private static MiniTezCluster miniTezCluster = null;
private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR
+ TestDAGRecovery2.class.getName() + "-tmpDir";
private static MiniDFSCluster dfsCluster = null;
private static TezClient tezSession = null;
private static FileSystem remoteFs = null;
@BeforeClass
public static void beforeClass() throws Exception {
LOG.info("Starting mini clusters");
try {
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
.format(true).racks(null).build();
remoteFs = dfsCluster.getFileSystem();
} catch (IOException io) {
throw new RuntimeException("problem starting mini dfs cluster", io);
}
if (miniTezCluster == null) {
miniTezCluster = new MiniTezCluster(TestDAGRecovery2.class.getName(),
1, 1, 1);
Configuration miniTezconf = new Configuration(conf);
miniTezconf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 4);
miniTezconf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
miniTezCluster.init(miniTezconf);
miniTezCluster.start();
}
}
@AfterClass
public static void afterClass() throws InterruptedException {
if (tezSession != null) {
try {
LOG.info("Stopping Tez Session");
tezSession.stop();
} catch (Exception e) {
e.printStackTrace();
}
}
Thread.sleep(10000);
if (miniTezCluster != null) {
try {
LOG.info("Stopping MiniTezCluster");
miniTezCluster.stop();
} catch (Exception e) {
e.printStackTrace();
}
}
if (dfsCluster != null) {
try {
LOG.info("Stopping DFSCluster");
dfsCluster.shutdown();
} catch (Exception e) {
e.printStackTrace();
}
}
}
private TezConfiguration createSessionConfig(Path remoteStagingDir) {
TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
tezConf.setInt(TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS, 10);
tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "DEBUG");
tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR,
remoteStagingDir.toString());
tezConf.setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, false);
tezConf.setInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, 4);
tezConf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 500);
tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m");
tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
return tezConf;
}
@Before
public void setup() throws Exception {
Path remoteStagingDir = remoteFs.makeQualified(new Path(TEST_ROOT_DIR, String
.valueOf(new Random().nextInt(100000))));
TezClientUtils.ensureStagingDirExists(conf, remoteStagingDir);
TezConfiguration tezConf = createSessionConfig(remoteStagingDir);
tezSession = TezClient.create("TestDAGRecovery2", tezConf);
tezSession.start();
}
@After
public void teardown() throws InterruptedException {
if (tezSession != null) {
try {
LOG.info("Stopping Tez Session");
tezSession.stop();
} catch (Exception e) {
e.printStackTrace();
}
}
tezSession = null;
Thread.sleep(10000);
}
void runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception {
runDAGAndVerify(dag, finalState, tezSession);
}
void runDAGAndVerify(DAG dag, DAGStatus.State finalState,
TezClient session) throws Exception {
session.waitTillReady();
DAGClient dagClient = session.submitDAG(dag);
DAGStatus dagStatus = dagClient.getDAGStatus(null);
while (!dagStatus.isCompleted()) {
LOG.info("Waiting for dag to complete. Sleeping for 500ms."
+ " DAG name: " + dag.getName()
+ " DAG appContext: " + dagClient.getExecutionContext()
+ " Current state: " + dagStatus.getState());
Thread.sleep(100);
dagStatus = dagClient.getDAGStatus(null);
}
Assert.assertEquals(finalState, dagStatus.getState());
}
@Test(timeout=120000)
public void testFailingCommitter() throws Exception {
DAG dag = SimpleVTestDAG.createDAG("FailingCommitterDAG", null);
OutputDescriptor od =
OutputDescriptor.create(MultiAttemptDAG.NoOpOutput.class.getName());
od.setUserPayload(UserPayload.create(ByteBuffer.wrap(
new MultiAttemptDAG.FailingOutputCommitter.FailingOutputCommitterConfig(true)
.toUserPayload())));
OutputCommitterDescriptor ocd = OutputCommitterDescriptor.create(
MultiAttemptDAG.FailingOutputCommitter.class.getName());
dag.getVertex("v3").addDataSink("FailingOutput", new DataSinkDescriptor(od, ocd, null));
runDAGAndVerify(dag, State.FAILED);
}
@Test(timeout=120000)
public void testSessionDisableMultiAttempts() throws Exception {
tezSession.stop();
Path remoteStagingDir = remoteFs.makeQualified(new Path(TEST_ROOT_DIR, String
.valueOf(new Random().nextInt(100000))));
TezClientUtils.ensureStagingDirExists(conf, remoteStagingDir);
TezConfiguration tezConf = createSessionConfig(remoteStagingDir);
tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
tezConf.setBoolean(TezConfiguration.DAG_RECOVERY_ENABLED, false);
TezClient session = TezClient.create("TestDAGRecovery2SingleAttemptOnly", tezConf);
session.start();
// DAG should fail as it never completes on the first attempt
DAG dag = MultiAttemptDAG.createDAG("TestSingleAttemptDAG", null);
runDAGAndVerify(dag, State.FAILED, session);
session.stop();
}
}