|
@@ -30,22 +30,17 @@ import java.util.Set;
|
|
import java.util.UUID;
|
|
import java.util.UUID;
|
|
|
|
|
|
import com.google.common.collect.Sets;
|
|
import com.google.common.collect.Sets;
|
|
-import org.junit.AfterClass;
|
|
|
|
-import org.junit.BeforeClass;
|
|
|
|
import org.junit.Rule;
|
|
import org.junit.Rule;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.junit.rules.TemporaryFolder;
|
|
import org.junit.rules.TemporaryFolder;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
-import org.apache.hadoop.fs.FileSystem;
|
|
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
|
import org.apache.hadoop.fs.s3a.S3AUtils;
|
|
import org.apache.hadoop.fs.s3a.S3AUtils;
|
|
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
|
|
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
|
|
-import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
|
import org.apache.hadoop.io.LongWritable;
|
|
import org.apache.hadoop.io.LongWritable;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.mapred.JobConf;
|
|
import org.apache.hadoop.mapred.JobConf;
|
|
@@ -54,98 +49,36 @@ import org.apache.hadoop.mapreduce.Mapper;
|
|
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
|
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
|
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
|
|
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
|
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
-import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
|
|
|
|
-import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
|
|
|
|
import org.apache.hadoop.util.DurationInfo;
|
|
import org.apache.hadoop.util.DurationInfo;
|
|
|
|
|
|
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
|
|
|
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.terminateService;
|
|
|
|
-import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
|
|
|
|
-import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.*;
|
|
|
|
|
|
+import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID;
|
|
|
|
+
|
|
|
|
+/**
|
|
|
|
+ * Test for an MR Job with all the different committers.
|
|
|
|
+ */
|
|
|
|
+public abstract class AbstractITCommitMRJob extends AbstractYarnClusterITest {
|
|
|
|
|
|
-/** Full integration test of an MR job. */
|
|
|
|
-public abstract class AbstractITCommitMRJob extends AbstractCommitITest {
|
|
|
|
private static final Logger LOG =
|
|
private static final Logger LOG =
|
|
LoggerFactory.getLogger(AbstractITCommitMRJob.class);
|
|
LoggerFactory.getLogger(AbstractITCommitMRJob.class);
|
|
|
|
|
|
- private static final int TEST_FILE_COUNT = 2;
|
|
|
|
- private static final int SCALE_TEST_FILE_COUNT = 20;
|
|
|
|
-
|
|
|
|
- private static MiniDFSClusterService hdfs;
|
|
|
|
- private static MiniMRYarnCluster yarn = null;
|
|
|
|
- private static JobConf conf = null;
|
|
|
|
- private boolean uniqueFilenames = false;
|
|
|
|
- private boolean scaleTest;
|
|
|
|
-
|
|
|
|
- protected static FileSystem getDFS() {
|
|
|
|
- return hdfs.getClusterFS();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @BeforeClass
|
|
|
|
- public static void setupClusters() throws IOException {
|
|
|
|
- // the HDFS and YARN clusters share the same configuration, so
|
|
|
|
- // the HDFS cluster binding is implicitly propagated to YARN
|
|
|
|
- conf = new JobConf();
|
|
|
|
- conf.setBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, false);
|
|
|
|
- conf.setLong(CommonConfigurationKeys.FS_DU_INTERVAL_KEY, Long.MAX_VALUE);
|
|
|
|
-
|
|
|
|
- hdfs = deployService(conf, new MiniDFSClusterService());
|
|
|
|
- yarn = deployService(conf,
|
|
|
|
- new MiniMRYarnCluster("ITCommitMRJob", 2));
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @SuppressWarnings("ThrowableNotThrown")
|
|
|
|
- @AfterClass
|
|
|
|
- public static void teardownClusters() throws IOException {
|
|
|
|
- conf = null;
|
|
|
|
- yarn = terminateService(yarn);
|
|
|
|
- hdfs = terminateService(hdfs);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public static MiniDFSCluster getHdfs() {
|
|
|
|
- return hdfs.getCluster();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public static FileSystem getLocalFS() {
|
|
|
|
- return hdfs.getLocalFS();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
@Rule
|
|
@Rule
|
|
public final TemporaryFolder temp = new TemporaryFolder();
|
|
public final TemporaryFolder temp = new TemporaryFolder();
|
|
|
|
|
|
- /**
|
|
|
|
- * The name of the committer as returned by
|
|
|
|
- * {@link AbstractS3ACommitter#getName()} and used for committer construction.
|
|
|
|
- */
|
|
|
|
- protected abstract String committerName();
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public void setup() throws Exception {
|
|
|
|
- super.setup();
|
|
|
|
- scaleTest = getTestPropertyBool(
|
|
|
|
- getConfiguration(),
|
|
|
|
- KEY_SCALE_TESTS_ENABLED,
|
|
|
|
- DEFAULT_SCALE_TESTS_ENABLED);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- protected int getTestTimeoutMillis() {
|
|
|
|
- return SCALE_TEST_TIMEOUT_SECONDS * 1000;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
@Test
|
|
@Test
|
|
public void testMRJob() throws Exception {
|
|
public void testMRJob() throws Exception {
|
|
|
|
+ describe("Run a simple MR Job");
|
|
|
|
+
|
|
S3AFileSystem fs = getFileSystem();
|
|
S3AFileSystem fs = getFileSystem();
|
|
// final dest is in S3A
|
|
// final dest is in S3A
|
|
- Path outputPath = path("testMRJob");
|
|
|
|
|
|
+ Path outputPath = path(getMethodName());
|
|
|
|
|
|
String commitUUID = UUID.randomUUID().toString();
|
|
String commitUUID = UUID.randomUUID().toString();
|
|
- String suffix = uniqueFilenames ? ("-" + commitUUID) : "";
|
|
|
|
|
|
+ String suffix = isUniqueFilenames() ? ("-" + commitUUID) : "";
|
|
int numFiles = getTestFileCount();
|
|
int numFiles = getTestFileCount();
|
|
List<String> expectedFiles = new ArrayList<>(numFiles);
|
|
List<String> expectedFiles = new ArrayList<>(numFiles);
|
|
Set<String> expectedKeys = Sets.newHashSet();
|
|
Set<String> expectedKeys = Sets.newHashSet();
|
|
for (int i = 0; i < numFiles; i += 1) {
|
|
for (int i = 0; i < numFiles; i += 1) {
|
|
- File file = temp.newFile(String.valueOf(i) + ".text");
|
|
|
|
|
|
+ File file = temp.newFile(i + ".text");
|
|
try (FileOutputStream out = new FileOutputStream(file)) {
|
|
try (FileOutputStream out = new FileOutputStream(file)) {
|
|
out.write(("file " + i).getBytes(StandardCharsets.UTF_8));
|
|
out.write(("file " + i).getBytes(StandardCharsets.UTF_8));
|
|
}
|
|
}
|
|
@@ -156,17 +89,8 @@ public abstract class AbstractITCommitMRJob extends AbstractCommitITest {
|
|
}
|
|
}
|
|
Collections.sort(expectedFiles);
|
|
Collections.sort(expectedFiles);
|
|
|
|
|
|
- Job mrJob = Job.getInstance(yarn.getConfig(), "test-committer-job");
|
|
|
|
|
|
+ Job mrJob = createJob();
|
|
JobConf jobConf = (JobConf) mrJob.getConfiguration();
|
|
JobConf jobConf = (JobConf) mrJob.getConfiguration();
|
|
- jobConf.setBoolean(FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES,
|
|
|
|
- uniqueFilenames);
|
|
|
|
-
|
|
|
|
-
|
|
|
|
- bindCommitter(jobConf,
|
|
|
|
- CommitConstants.S3A_COMMITTER_FACTORY,
|
|
|
|
- committerName());
|
|
|
|
- // pass down the scale test flag
|
|
|
|
- jobConf.setBoolean(KEY_SCALE_TESTS_ENABLED, scaleTest);
|
|
|
|
|
|
|
|
mrJob.setOutputFormatClass(LoggingTextOutputFormat.class);
|
|
mrJob.setOutputFormatClass(LoggingTextOutputFormat.class);
|
|
FileOutputFormat.setOutputPath(mrJob, outputPath);
|
|
FileOutputFormat.setOutputPath(mrJob, outputPath);
|
|
@@ -200,7 +124,7 @@ public abstract class AbstractITCommitMRJob extends AbstractCommitITest {
|
|
mrJob.setMaxMapAttempts(1);
|
|
mrJob.setMaxMapAttempts(1);
|
|
|
|
|
|
mrJob.submit();
|
|
mrJob.submit();
|
|
- try (DurationInfo d = new DurationInfo(LOG, "Job Execution")) {
|
|
|
|
|
|
+ try (DurationInfo ignore = new DurationInfo(LOG, "Job Execution")) {
|
|
boolean succeeded = mrJob.waitForCompletion(true);
|
|
boolean succeeded = mrJob.waitForCompletion(true);
|
|
assertTrue("MR job failed", succeeded);
|
|
assertTrue("MR job failed", succeeded);
|
|
}
|
|
}
|
|
@@ -219,24 +143,11 @@ public abstract class AbstractITCommitMRJob extends AbstractCommitITest {
|
|
}
|
|
}
|
|
Collections.sort(actualFiles);
|
|
Collections.sort(actualFiles);
|
|
|
|
|
|
- // load in the success data marker: this guarantees that a s3guard
|
|
|
|
- // committer was used
|
|
|
|
- Path success = new Path(outputPath, _SUCCESS);
|
|
|
|
- FileStatus status = fs.getFileStatus(success);
|
|
|
|
- assertTrue("0 byte success file - not a s3guard committer " + success,
|
|
|
|
- status.getLen() > 0);
|
|
|
|
- SuccessData successData = SuccessData.load(fs, success);
|
|
|
|
- String commitDetails = successData.toString();
|
|
|
|
- LOG.info("Committer name " + committerName() + "\n{}",
|
|
|
|
- commitDetails);
|
|
|
|
- LOG.info("Committer statistics: \n{}",
|
|
|
|
- successData.dumpMetrics(" ", " = ", "\n"));
|
|
|
|
- LOG.info("Diagnostics\n{}",
|
|
|
|
- successData.dumpDiagnostics(" ", " = ", "\n"));
|
|
|
|
- assertEquals("Wrong committer in " + commitDetails,
|
|
|
|
- committerName(), successData.getCommitter());
|
|
|
|
|
|
+ SuccessData successData = validateSuccessFile(fs, outputPath,
|
|
|
|
+ committerName());
|
|
List<String> successFiles = successData.getFilenames();
|
|
List<String> successFiles = successData.getFilenames();
|
|
- assertTrue("No filenames in " + commitDetails,
|
|
|
|
|
|
+ String commitData = successData.toString();
|
|
|
|
+ assertTrue("No filenames in " + commitData,
|
|
!successFiles.isEmpty());
|
|
!successFiles.isEmpty());
|
|
|
|
|
|
assertEquals("Should commit the expected files",
|
|
assertEquals("Should commit the expected files",
|
|
@@ -245,41 +156,12 @@ public abstract class AbstractITCommitMRJob extends AbstractCommitITest {
|
|
Set<String> summaryKeys = Sets.newHashSet();
|
|
Set<String> summaryKeys = Sets.newHashSet();
|
|
summaryKeys.addAll(successFiles);
|
|
summaryKeys.addAll(successFiles);
|
|
assertEquals("Summary keyset doesn't list the the expected paths "
|
|
assertEquals("Summary keyset doesn't list the the expected paths "
|
|
- + commitDetails, expectedKeys, summaryKeys);
|
|
|
|
|
|
+ + commitData, expectedKeys, summaryKeys);
|
|
assertPathDoesNotExist("temporary dir",
|
|
assertPathDoesNotExist("temporary dir",
|
|
new Path(outputPath, CommitConstants.TEMPORARY));
|
|
new Path(outputPath, CommitConstants.TEMPORARY));
|
|
customPostExecutionValidation(outputPath, successData);
|
|
customPostExecutionValidation(outputPath, successData);
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Get the file count for the test.
|
|
|
|
- * @return the number of mappers to create.
|
|
|
|
- */
|
|
|
|
- public int getTestFileCount() {
|
|
|
|
- return scaleTest ? SCALE_TEST_FILE_COUNT : TEST_FILE_COUNT;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Override point to let implementations tune the MR Job conf.
|
|
|
|
- * @param jobConf configuration
|
|
|
|
- */
|
|
|
|
- protected void applyCustomConfigOptions(JobConf jobConf) throws IOException {
|
|
|
|
-
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Override point for any committer specific validation operations;
|
|
|
|
- * called after the base assertions have all passed.
|
|
|
|
- * @param destPath destination of work
|
|
|
|
- * @param successData loaded success data
|
|
|
|
- * @throws Exception failure
|
|
|
|
- */
|
|
|
|
- protected void customPostExecutionValidation(Path destPath,
|
|
|
|
- SuccessData successData)
|
|
|
|
- throws Exception {
|
|
|
|
-
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Test Mapper.
|
|
* Test Mapper.
|
|
* This is executed in separate process, and must not make any assumptions
|
|
* This is executed in separate process, and must not make any assumptions
|
|
@@ -301,7 +183,7 @@ public abstract class AbstractITCommitMRJob extends AbstractCommitITest {
|
|
org.apache.log4j.BasicConfigurator.configure();
|
|
org.apache.log4j.BasicConfigurator.configure();
|
|
boolean scaleMap = context.getConfiguration()
|
|
boolean scaleMap = context.getConfiguration()
|
|
.getBoolean(KEY_SCALE_TESTS_ENABLED, false);
|
|
.getBoolean(KEY_SCALE_TESTS_ENABLED, false);
|
|
- operations = scaleMap ? 1000 : 10;
|
|
|
|
|
|
+ operations = scaleMap ? SCALE_TEST_KEYS : BASE_TEST_KEYS;
|
|
id = context.getTaskAttemptID().toString();
|
|
id = context.getTaskAttemptID().toString();
|
|
}
|
|
}
|
|
|
|
|