MAPREDUCE-2765. DistCp Rewrite. (Mithun Radhakrishnan via mahadev)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1236049 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
10a3c56fcf
commit
505dd18690
|
@ -130,6 +130,8 @@ Release 0.23.1 - Unreleased
|
|||
MAPREDUCE-3710. Improved FileInputFormat to return better locality for the
|
||||
last split. (Siddarth Seth via vinodkv)
|
||||
|
||||
MAPREDUCE-2765. DistCp Rewrite. (Mithun Radhakrishnan via mahadev)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
MAPREDUCE-3567. Extraneous JobConf objects in AM heap. (Vinod Kumar
|
||||
|
|
|
@ -714,11 +714,21 @@
|
|||
<artifactId>maven-project-info-reports-plugin</artifactId>
|
||||
<version>2.4</version>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-resources-plugin</artifactId>
|
||||
<version>2.2</version>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>exec-maven-plugin</artifactId>
|
||||
<version>1.2</version>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-pdf-plugin</artifactId>
|
||||
<version>1.1</version>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</pluginManagement>
|
||||
|
||||
|
@ -816,6 +826,14 @@
|
|||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-pdf-plugin</artifactId>
|
||||
<configuration>
|
||||
<outputDirectory>${project.reporting.outputDirectory}</outputDirectory>
|
||||
<includeReports>false</includeReports>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
|
|
|
@ -0,0 +1,7 @@
|
|||
DistCp (distributed copy) is a tool used for large inter/intra-cluster copying.
|
||||
It uses Map/Reduce to effect its distribution, error handling and recovery,
|
||||
and reporting. It expands a list of files and directories into input to map tasks,
|
||||
each of which will copy a partition of the files specified in the source list.
|
||||
|
||||
Version 0.1 (2010/08/02 sriksun)
|
||||
- Initial Version
|
|
@ -0,0 +1,185 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-project</artifactId>
|
||||
<version>0.23.1-SNAPSHOT</version>
|
||||
<relativePath>../../hadoop-project</relativePath>
|
||||
</parent>
|
||||
<groupId>org.apache.hadoop.tools</groupId>
|
||||
<artifactId>hadoop-distcp</artifactId>
|
||||
<version>0.23.1-SNAPSHOT</version>
|
||||
<description>Apache Hadoop Distributed Copy</description>
|
||||
<name>Apache Hadoop Distributed Copy</name>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<file.encoding>UTF-8</file.encoding>
|
||||
<downloadSources>true</downloadSources>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-annotations</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-app</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-hs</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-core</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-jobclient</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-jobclient</artifactId>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs</artifactId>
|
||||
<scope>provided</scope>
|
||||
</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-common</artifactId>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<resources>
|
||||
<resource>
|
||||
<directory>src/main/resources</directory>
|
||||
<filtering>true</filtering>
|
||||
</resource>
|
||||
</resources>
|
||||
<testResources>
|
||||
<testResource>
|
||||
<directory>src/test/resources</directory>
|
||||
<filtering>true</filtering>
|
||||
</testResource>
|
||||
</testResources>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<configuration>
|
||||
<forkMode>always</forkMode>
|
||||
<forkedProcessTimeoutInSeconds>600</forkedProcessTimeoutInSeconds>
|
||||
<argLine>-Xmx1024m</argLine>
|
||||
<includes>
|
||||
<include>**/Test*.java</include>
|
||||
</includes>
|
||||
<redirectTestOutputToFile>true</redirectTestOutputToFile>
|
||||
<systemProperties>
|
||||
<property>
|
||||
<name>test.build.data</name>
|
||||
<value>${basedir}/target/test/data</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hadoop.log.dir</name>
|
||||
<value>target/test/logs</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>org.apache.commons.logging.Log</name>
|
||||
<value>org.apache.commons.logging.impl.SimpleLog</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>org.apache.commons.logging.simplelog.defaultlog</name>
|
||||
<value>warn</value>
|
||||
</property>
|
||||
</systemProperties>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<phase>package</phase>
|
||||
<goals>
|
||||
<goal>copy-dependencies</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<outputDirectory>${project.build.directory}/lib</outputDirectory>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-checkstyle-plugin</artifactId>
|
||||
<configuration>
|
||||
<enableRulesSummary>true</enableRulesSummary>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<configuration>
|
||||
<archive>
|
||||
<manifest>
|
||||
<mainClass>org.apache.hadoop.tools.DistCp</mainClass>
|
||||
</manifest>
|
||||
</archive>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-source-plugin</artifactId>
|
||||
<configuration>
|
||||
<attach>true</attach>
|
||||
</configuration>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>jar</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-pdf-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>pdf</id>
|
||||
<phase>package</phase>
|
||||
<goals>
|
||||
<goal>pdf</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
|
@ -0,0 +1,218 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* The CopyListing abstraction is responsible for how the list of
|
||||
* sources and targets is constructed, for DistCp's copy function.
|
||||
* The copy-listing should be a SequenceFile<Text, FileStatus>,
|
||||
* located at the path specified to buildListing(),
|
||||
* each entry being a pair of (Source relative path, source file status),
|
||||
* all the paths being fully qualified.
|
||||
*/
|
||||
public abstract class CopyListing extends Configured {
|
||||
|
||||
private Credentials credentials;
|
||||
|
||||
/**
|
||||
* Build listing function creates the input listing that distcp uses to
|
||||
* perform the copy.
|
||||
*
|
||||
* The build listing is a sequence file that has relative path of a file in the key
|
||||
* and the file status information of the source file in the value
|
||||
*
|
||||
* For instance if the source path is /tmp/data and the traversed path is
|
||||
* /tmp/data/dir1/dir2/file1, then the sequence file would contain
|
||||
*
|
||||
* key: /dir1/dir2/file1 and value: FileStatus(/tmp/data/dir1/dir2/file1)
|
||||
*
|
||||
* File would also contain directory entries. Meaning, if /tmp/data/dir1/dir2/file1
|
||||
* is the only file under /tmp/data, the resulting sequence file would contain the
|
||||
* following entries
|
||||
*
|
||||
* key: /dir1 and value: FileStatus(/tmp/data/dir1)
|
||||
* key: /dir1/dir2 and value: FileStatus(/tmp/data/dir1/dir2)
|
||||
* key: /dir1/dir2/file1 and value: FileStatus(/tmp/data/dir1/dir2/file1)
|
||||
*
|
||||
* Cases requiring special handling:
|
||||
* If source path is a file (/tmp/file1), contents of the file will be as follows
|
||||
*
|
||||
* TARGET DOES NOT EXIST: Key-"", Value-FileStatus(/tmp/file1)
|
||||
* TARGET IS FILE : Key-"", Value-FileStatus(/tmp/file1)
|
||||
* TARGET IS DIR : Key-"/file1", Value-FileStatus(/tmp/file1)
|
||||
*
|
||||
* @param pathToListFile - Output file where the listing would be stored
|
||||
* @param options - Input options to distcp
|
||||
* @throws IOException - Exception if any
|
||||
*/
|
||||
public final void buildListing(Path pathToListFile,
|
||||
DistCpOptions options) throws IOException {
|
||||
validatePaths(options);
|
||||
doBuildListing(pathToListFile, options);
|
||||
Configuration config = getConf();
|
||||
|
||||
config.set(DistCpConstants.CONF_LABEL_LISTING_FILE_PATH, pathToListFile.toString());
|
||||
config.setLong(DistCpConstants.CONF_LABEL_TOTAL_BYTES_TO_BE_COPIED, getBytesToCopy());
|
||||
config.setLong(DistCpConstants.CONF_LABEL_TOTAL_NUMBER_OF_RECORDS, getNumberOfPaths());
|
||||
|
||||
checkForDuplicates(pathToListFile);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate input and output paths
|
||||
*
|
||||
* @param options - Input options
|
||||
* @throws InvalidInputException: If inputs are invalid
|
||||
* @throws IOException: any Exception with FS
|
||||
*/
|
||||
protected abstract void validatePaths(DistCpOptions options)
|
||||
throws IOException, InvalidInputException;
|
||||
|
||||
/**
|
||||
* The interface to be implemented by sub-classes, to create the source/target file listing.
|
||||
* @param pathToListFile Path on HDFS where the listing file is written.
|
||||
* @param options Input Options for DistCp (indicating source/target paths.)
|
||||
* @throws IOException: Thrown on failure to create the listing file.
|
||||
*/
|
||||
protected abstract void doBuildListing(Path pathToListFile,
|
||||
DistCpOptions options) throws IOException;
|
||||
|
||||
/**
|
||||
* Return the total bytes that distCp should copy for the source paths
|
||||
* This doesn't consider whether file is same should be skipped during copy
|
||||
*
|
||||
* @return total bytes to copy
|
||||
*/
|
||||
protected abstract long getBytesToCopy();
|
||||
|
||||
/**
|
||||
* Return the total number of paths to distcp, includes directories as well
|
||||
* This doesn't consider whether file/dir is already present and should be skipped during copy
|
||||
*
|
||||
* @return Total number of paths to distcp
|
||||
*/
|
||||
protected abstract long getNumberOfPaths();
|
||||
|
||||
/**
|
||||
* Validate the final resulting path listing to see if there are any duplicate entries
|
||||
*
|
||||
* @param pathToListFile - path listing build by doBuildListing
|
||||
* @throws IOException - Any issues while checking for duplicates and throws
|
||||
* @throws DuplicateFileException - if there are duplicates
|
||||
*/
|
||||
private void checkForDuplicates(Path pathToListFile)
|
||||
throws DuplicateFileException, IOException {
|
||||
|
||||
Configuration config = getConf();
|
||||
FileSystem fs = pathToListFile.getFileSystem(config);
|
||||
|
||||
Path sortedList = DistCpUtils.sortListing(fs, config, pathToListFile);
|
||||
|
||||
SequenceFile.Reader reader = new SequenceFile.Reader(
|
||||
config, SequenceFile.Reader.file(sortedList));
|
||||
try {
|
||||
Text lastKey = new Text("*"); //source relative path can never hold *
|
||||
FileStatus lastFileStatus = new FileStatus();
|
||||
|
||||
Text currentKey = new Text();
|
||||
while (reader.next(currentKey)) {
|
||||
if (currentKey.equals(lastKey)) {
|
||||
FileStatus currentFileStatus = new FileStatus();
|
||||
reader.getCurrentValue(currentFileStatus);
|
||||
throw new DuplicateFileException("File " + lastFileStatus.getPath() + " and " +
|
||||
currentFileStatus.getPath() + " would cause duplicates. Aborting");
|
||||
}
|
||||
reader.getCurrentValue(lastFileStatus);
|
||||
lastKey.set(currentKey);
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeStream(reader);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Protected constructor, to initialize configuration.
|
||||
* @param configuration The input configuration,
|
||||
* with which the source/target FileSystems may be accessed.
|
||||
* @param credentials - Credentials object on which the FS delegation tokens are cached.If null
|
||||
* delegation token caching is skipped
|
||||
*/
|
||||
protected CopyListing(Configuration configuration, Credentials credentials) {
|
||||
setConf(configuration);
|
||||
setCredentials(credentials);
|
||||
}
|
||||
|
||||
/**
|
||||
* set Credentials store, on which FS delegatin token will be cached
|
||||
* @param credentials - Credentials object
|
||||
*/
|
||||
protected void setCredentials(Credentials credentials) {
|
||||
this.credentials = credentials;
|
||||
}
|
||||
|
||||
/**
|
||||
* get credentials to update the delegation tokens for accessed FS objects
|
||||
* @return Credentials object
|
||||
*/
|
||||
protected Credentials getCredentials() {
|
||||
return credentials;
|
||||
}
|
||||
|
||||
/**
|
||||
* Public Factory method with which the appropriate CopyListing implementation may be retrieved.
|
||||
* @param configuration The input configuration.
|
||||
* @param credentials Credentials object on which the FS delegation tokens are cached
|
||||
* @param options The input Options, to help choose the appropriate CopyListing Implementation.
|
||||
* @return An instance of the appropriate CopyListing implementation.
|
||||
*/
|
||||
public static CopyListing getCopyListing(Configuration configuration,
|
||||
Credentials credentials,
|
||||
DistCpOptions options) {
|
||||
if (options.getSourceFileListing() == null) {
|
||||
return new GlobbedCopyListing(configuration, credentials);
|
||||
} else {
|
||||
return new FileBasedCopyListing(configuration, credentials);
|
||||
}
|
||||
}
|
||||
|
||||
static class DuplicateFileException extends RuntimeException {
|
||||
public DuplicateFileException(String message) {
|
||||
super(message);
|
||||
}
|
||||
}
|
||||
|
||||
static class InvalidInputException extends RuntimeException {
|
||||
public InvalidInputException(String message) {
|
||||
super(message);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,405 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hadoop.mapreduce.JobSubmissionFiles;
|
||||
import org.apache.hadoop.mapreduce.Cluster;
|
||||
import org.apache.hadoop.tools.CopyListing.*;
|
||||
import org.apache.hadoop.tools.mapred.CopyMapper;
|
||||
import org.apache.hadoop.tools.mapred.CopyOutputFormat;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Random;
|
||||
|
||||
/**
|
||||
* DistCp is the main driver-class for DistCpV2.
|
||||
* For command-line use, DistCp::main() orchestrates the parsing of command-line
|
||||
* parameters and the launch of the DistCp job.
|
||||
* For programmatic use, a DistCp object can be constructed by specifying
|
||||
* options (in a DistCpOptions object), and DistCp::execute() may be used to
|
||||
* launch the copy-job. DistCp may alternatively be sub-classed to fine-tune
|
||||
* behaviour.
|
||||
*/
|
||||
public class DistCp extends Configured implements Tool {
|
||||
private static final Log LOG = LogFactory.getLog(DistCp.class);
|
||||
|
||||
private DistCpOptions inputOptions;
|
||||
private Path metaFolder;
|
||||
|
||||
private static final String PREFIX = "_distcp";
|
||||
private static final String WIP_PREFIX = "._WIP_";
|
||||
private static final String DISTCP_DEFAULT_XML = "distcp-default.xml";
|
||||
public static final Random rand = new Random();
|
||||
|
||||
private boolean submitted;
|
||||
private FileSystem jobFS;
|
||||
|
||||
/**
|
||||
* Public Constructor. Creates DistCp object with specified input-parameters.
|
||||
* (E.g. source-paths, target-location, etc.)
|
||||
* @param inputOptions Options (indicating source-paths, target-location.)
|
||||
* @param configuration The Hadoop configuration against which the Copy-mapper must run.
|
||||
* @throws Exception, on failure.
|
||||
*/
|
||||
public DistCp(Configuration configuration, DistCpOptions inputOptions) throws Exception {
|
||||
Configuration config = new Configuration(configuration);
|
||||
config.addResource(DISTCP_DEFAULT_XML);
|
||||
setConf(config);
|
||||
this.inputOptions = inputOptions;
|
||||
this.metaFolder = createMetaFolderPath();
|
||||
}
|
||||
|
||||
/**
|
||||
* To be used with the ToolRunner. Not for public consumption.
|
||||
*/
|
||||
private DistCp() {}
|
||||
|
||||
/**
|
||||
* Implementation of Tool::run(). Orchestrates the copy of source file(s)
|
||||
* to target location, by:
|
||||
* 1. Creating a list of files to be copied to target.
|
||||
* 2. Launching a Map-only job to copy the files. (Delegates to execute().)
|
||||
* @param argv List of arguments passed to DistCp, from the ToolRunner.
|
||||
* @return On success, it returns 0. Else, -1.
|
||||
*/
|
||||
public int run(String[] argv) {
|
||||
try {
|
||||
inputOptions = (OptionsParser.parse(argv));
|
||||
|
||||
LOG.info("Input Options: " + inputOptions);
|
||||
} catch (Throwable e) {
|
||||
LOG.error("Invalid arguments: ", e);
|
||||
System.err.println("Invalid arguments: " + e.getMessage());
|
||||
OptionsParser.usage();
|
||||
return DistCpConstants.INVALID_ARGUMENT;
|
||||
}
|
||||
|
||||
try {
|
||||
execute();
|
||||
} catch (InvalidInputException e) {
|
||||
LOG.error("Invalid input: ", e);
|
||||
return DistCpConstants.INVALID_ARGUMENT;
|
||||
} catch (DuplicateFileException e) {
|
||||
LOG.error("Duplicate files in input path: ", e);
|
||||
return DistCpConstants.DUPLICATE_INPUT;
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
return DistCpConstants.UNKNOWN_ERROR;
|
||||
}
|
||||
return DistCpConstants.SUCCESS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Implements the core-execution. Creates the file-list for copy,
|
||||
* and launches the Hadoop-job, to do the copy.
|
||||
* @return Job handle
|
||||
* @throws Exception, on failure.
|
||||
*/
|
||||
public Job execute() throws Exception {
|
||||
assert inputOptions != null;
|
||||
assert getConf() != null;
|
||||
|
||||
Job job = null;
|
||||
try {
|
||||
metaFolder = createMetaFolderPath();
|
||||
jobFS = metaFolder.getFileSystem(getConf());
|
||||
|
||||
job = createJob();
|
||||
createInputFileListing(job);
|
||||
|
||||
job.submit();
|
||||
submitted = true;
|
||||
} finally {
|
||||
if (!submitted) {
|
||||
cleanup();
|
||||
}
|
||||
}
|
||||
|
||||
String jobID = job.getJobID().toString();
|
||||
job.getConfiguration().set(DistCpConstants.CONF_LABEL_DISTCP_JOB_ID, jobID);
|
||||
|
||||
LOG.info("DistCp job-id: " + jobID);
|
||||
if (inputOptions.shouldBlock()) {
|
||||
job.waitForCompletion(true);
|
||||
}
|
||||
return job;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create Job object for submitting it, with all the configuration
|
||||
*
|
||||
* @return Reference to job object.
|
||||
* @throws IOException - Exception if any
|
||||
*/
|
||||
private Job createJob() throws IOException {
|
||||
String jobName = "distcp";
|
||||
String userChosenName = getConf().get(JobContext.JOB_NAME);
|
||||
if (userChosenName != null)
|
||||
jobName += ": " + userChosenName;
|
||||
Job job = Job.getInstance(getConf());
|
||||
job.setJobName(jobName);
|
||||
job.setInputFormatClass(DistCpUtils.getStrategy(getConf(), inputOptions));
|
||||
job.setJarByClass(CopyMapper.class);
|
||||
configureOutputFormat(job);
|
||||
|
||||
job.setMapperClass(CopyMapper.class);
|
||||
job.setNumReduceTasks(0);
|
||||
job.setMapOutputKeyClass(Text.class);
|
||||
job.setMapOutputValueClass(Text.class);
|
||||
job.setOutputFormatClass(CopyOutputFormat.class);
|
||||
job.getConfiguration().set(JobContext.MAP_SPECULATIVE, "false");
|
||||
job.getConfiguration().set(JobContext.NUM_MAPS,
|
||||
String.valueOf(inputOptions.getMaxMaps()));
|
||||
|
||||
if (inputOptions.getSslConfigurationFile() != null) {
|
||||
setupSSLConfig(job);
|
||||
}
|
||||
|
||||
inputOptions.appendToConf(job.getConfiguration());
|
||||
return job;
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup ssl configuration on the job configuration to enable hsftp access
|
||||
* from map job. Also copy the ssl configuration file to Distributed cache
|
||||
*
|
||||
* @param job - Reference to job's handle
|
||||
* @throws java.io.IOException - Exception if unable to locate ssl config file
|
||||
*/
|
||||
private void setupSSLConfig(Job job) throws IOException {
|
||||
Configuration configuration = job.getConfiguration();
|
||||
Path sslConfigPath = new Path(configuration.
|
||||
getResource(inputOptions.getSslConfigurationFile()).toString());
|
||||
|
||||
addSSLFilesToDistCache(job, sslConfigPath);
|
||||
configuration.set(DistCpConstants.CONF_LABEL_SSL_CONF, sslConfigPath.getName());
|
||||
configuration.set(DistCpConstants.CONF_LABEL_SSL_KEYSTORE, sslConfigPath.getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Add SSL files to distributed cache. Trust store, key store and ssl config xml
|
||||
*
|
||||
* @param job - Job handle
|
||||
* @param sslConfigPath - ssl Configuration file specified through options
|
||||
* @throws IOException - If any
|
||||
*/
|
||||
private void addSSLFilesToDistCache(Job job,
|
||||
Path sslConfigPath) throws IOException {
|
||||
Configuration configuration = job.getConfiguration();
|
||||
FileSystem localFS = FileSystem.getLocal(configuration);
|
||||
|
||||
Configuration sslConf = new Configuration(false);
|
||||
sslConf.addResource(sslConfigPath);
|
||||
|
||||
Path localStorePath = getLocalStorePath(sslConf,
|
||||
DistCpConstants.CONF_LABEL_SSL_TRUST_STORE_LOCATION);
|
||||
job.addCacheFile(localStorePath.makeQualified(localFS.getUri(),
|
||||
localFS.getWorkingDirectory()).toUri());
|
||||
configuration.set(DistCpConstants.CONF_LABEL_SSL_TRUST_STORE_LOCATION,
|
||||
localStorePath.getName());
|
||||
|
||||
localStorePath = getLocalStorePath(sslConf,
|
||||
DistCpConstants.CONF_LABEL_SSL_KEY_STORE_LOCATION);
|
||||
job.addCacheFile(localStorePath.makeQualified(localFS.getUri(),
|
||||
localFS.getWorkingDirectory()).toUri());
|
||||
configuration.set(DistCpConstants.CONF_LABEL_SSL_KEY_STORE_LOCATION,
|
||||
localStorePath.getName());
|
||||
|
||||
job.addCacheFile(sslConfigPath.makeQualified(localFS.getUri(),
|
||||
localFS.getWorkingDirectory()).toUri());
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Local Trust store/key store path
|
||||
*
|
||||
* @param sslConf - Config from SSL Client xml
|
||||
* @param storeKey - Key for either trust store or key store
|
||||
* @return - Path where the store is present
|
||||
* @throws IOException -If any
|
||||
*/
|
||||
private Path getLocalStorePath(Configuration sslConf, String storeKey) throws IOException {
|
||||
if (sslConf.get(storeKey) != null) {
|
||||
return new Path(sslConf.get(storeKey));
|
||||
} else {
|
||||
throw new IOException("Store for " + storeKey + " is not set in " +
|
||||
inputOptions.getSslConfigurationFile());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup output format appropriately
|
||||
*
|
||||
* @param job - Job handle
|
||||
* @throws IOException - Exception if any
|
||||
*/
|
||||
private void configureOutputFormat(Job job) throws IOException {
|
||||
final Configuration configuration = job.getConfiguration();
|
||||
Path targetPath = inputOptions.getTargetPath();
|
||||
FileSystem targetFS = targetPath.getFileSystem(configuration);
|
||||
targetPath = targetPath.makeQualified(targetFS.getUri(),
|
||||
targetFS.getWorkingDirectory());
|
||||
|
||||
if (inputOptions.shouldAtomicCommit()) {
|
||||
Path workDir = inputOptions.getAtomicWorkPath();
|
||||
if (workDir == null) {
|
||||
workDir = targetPath.getParent();
|
||||
}
|
||||
workDir = new Path(workDir, WIP_PREFIX + targetPath.getName()
|
||||
+ rand.nextInt());
|
||||
FileSystem workFS = workDir.getFileSystem(configuration);
|
||||
if (!DistCpUtils.compareFs(targetFS, workFS)) {
|
||||
throw new IllegalArgumentException("Work path " + workDir +
|
||||
" and target path " + targetPath + " are in different file system");
|
||||
}
|
||||
CopyOutputFormat.setWorkingDirectory(job, workDir);
|
||||
} else {
|
||||
CopyOutputFormat.setWorkingDirectory(job, targetPath);
|
||||
}
|
||||
CopyOutputFormat.setCommitDirectory(job, targetPath);
|
||||
|
||||
Path logPath = inputOptions.getLogPath();
|
||||
if (logPath == null) {
|
||||
logPath = new Path(metaFolder, "_logs");
|
||||
} else {
|
||||
LOG.info("DistCp job log path: " + logPath);
|
||||
}
|
||||
CopyOutputFormat.setOutputPath(job, logPath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create input listing by invoking an appropriate copy listing
|
||||
* implementation. Also add delegation tokens for each path
|
||||
* to job's credential store
|
||||
*
|
||||
* @param job - Handle to job
|
||||
* @return Returns the path where the copy listing is created
|
||||
* @throws IOException - If any
|
||||
*/
|
||||
private Path createInputFileListing(Job job) throws IOException {
|
||||
Path fileListingPath = getFileListingPath();
|
||||
CopyListing copyListing = CopyListing.getCopyListing(job.getConfiguration(),
|
||||
job.getCredentials(), inputOptions);
|
||||
copyListing.buildListing(fileListingPath, inputOptions);
|
||||
return fileListingPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get default name of the copy listing file. Use the meta folder
|
||||
* to create the copy listing file
|
||||
*
|
||||
* @return - Path where the copy listing file has to be saved
|
||||
* @throws IOException - Exception if any
|
||||
*/
|
||||
private Path getFileListingPath() throws IOException {
|
||||
String fileListPathStr = metaFolder + "/fileList.seq";
|
||||
Path path = new Path(fileListPathStr);
|
||||
return new Path(path.toUri().normalize().toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a default working folder for the job, under the
|
||||
* job staging directory
|
||||
*
|
||||
* @return Returns the working folder information
|
||||
* @throws Exception - EXception if any
|
||||
*/
|
||||
private Path createMetaFolderPath() throws Exception {
|
||||
Configuration configuration = getConf();
|
||||
Path stagingDir = JobSubmissionFiles.getStagingDir(
|
||||
new Cluster(configuration), configuration);
|
||||
Path metaFolderPath = new Path(stagingDir, PREFIX + String.valueOf(rand.nextInt()));
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug("Meta folder location: " + metaFolderPath);
|
||||
configuration.set(DistCpConstants.CONF_LABEL_META_FOLDER, metaFolderPath.toString());
|
||||
return metaFolderPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Main function of the DistCp program. Parses the input arguments (via OptionsParser),
|
||||
* and invokes the DistCp::run() method, via the ToolRunner.
|
||||
* @param argv Command-line arguments sent to DistCp.
|
||||
*/
|
||||
public static void main(String argv[]) {
|
||||
try {
|
||||
DistCp distCp = new DistCp();
|
||||
Cleanup CLEANUP = new Cleanup(distCp);
|
||||
|
||||
Runtime.getRuntime().addShutdownHook(CLEANUP);
|
||||
System.exit(ToolRunner.run(getDefaultConf(), distCp, argv));
|
||||
}
|
||||
catch (Exception e) {
|
||||
LOG.error("Couldn't complete DistCp operation: ", e);
|
||||
System.exit(DistCpConstants.UNKNOWN_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads properties from distcp-default.xml into configuration
|
||||
* object
|
||||
* @return Configuration which includes properties from distcp-default.xml
|
||||
*/
|
||||
private static Configuration getDefaultConf() {
|
||||
Configuration config = new Configuration();
|
||||
config.addResource(DISTCP_DEFAULT_XML);
|
||||
return config;
|
||||
}
|
||||
|
||||
private synchronized void cleanup() {
|
||||
try {
|
||||
if (metaFolder == null) return;
|
||||
|
||||
jobFS.delete(metaFolder, true);
|
||||
metaFolder = null;
|
||||
} catch (IOException e) {
|
||||
LOG.error("Unable to cleanup meta folder: " + metaFolder, e);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isSubmitted() {
|
||||
return submitted;
|
||||
}
|
||||
|
||||
private static class Cleanup extends Thread {
|
||||
private final DistCp distCp;
|
||||
|
||||
public Cleanup(DistCp distCp) {
|
||||
this.distCp = distCp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
if (distCp.isSubmitted()) return;
|
||||
|
||||
distCp.cleanup();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,104 @@
|
|||
package org.apache.hadoop.tools;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Utility class to hold commonly used constants.
|
||||
*/
|
||||
public class DistCpConstants {
|
||||
|
||||
/* Default number of maps to use for DistCp */
|
||||
public static final int DEFAULT_MAPS = 20;
|
||||
|
||||
/* Default bandwidth if none specified */
|
||||
public static final int DEFAULT_BANDWIDTH_MB = 100;
|
||||
|
||||
/* Default strategy for copying. Implementation looked up
|
||||
from distcp-default.xml
|
||||
*/
|
||||
public static final String UNIFORMSIZE = "uniformsize";
|
||||
|
||||
/**
|
||||
* Constants mapping to command line switches/input options
|
||||
*/
|
||||
public static final String CONF_LABEL_ATOMIC_COPY = "distcp.atomic.copy";
|
||||
public static final String CONF_LABEL_WORK_PATH = "distcp.work.path";
|
||||
public static final String CONF_LABEL_LOG_PATH = "distcp.log.path";
|
||||
public static final String CONF_LABEL_IGNORE_FAILURES = "distcp.ignore.failures";
|
||||
public static final String CONF_LABEL_PRESERVE_STATUS = "distcp.preserve.status";
|
||||
public static final String CONF_LABEL_SYNC_FOLDERS = "distcp.sync.folders";
|
||||
public static final String CONF_LABEL_DELETE_MISSING = "distcp.delete.missing.source";
|
||||
public static final String CONF_LABEL_SSL_CONF = "distcp.keystore.resource";
|
||||
public static final String CONF_LABEL_MAX_MAPS = "distcp.max.maps";
|
||||
public static final String CONF_LABEL_SOURCE_LISTING = "distcp.source.listing";
|
||||
public static final String CONF_LABEL_COPY_STRATEGY = "distcp.copy.strategy";
|
||||
public static final String CONF_LABEL_SKIP_CRC = "distcp.skip.crc";
|
||||
public static final String CONF_LABEL_OVERWRITE = "distcp.copy.overwrite";
|
||||
public static final String CONF_LABEL_BANDWIDTH_MB = "distcp.map.bandwidth.mb";
|
||||
|
||||
/* Total bytes to be copied. Updated by copylisting. Unfiltered count */
|
||||
public static final String CONF_LABEL_TOTAL_BYTES_TO_BE_COPIED = "mapred.total.bytes.expected";
|
||||
|
||||
/* Total number of paths to copy, includes directories. Unfiltered count */
|
||||
public static final String CONF_LABEL_TOTAL_NUMBER_OF_RECORDS = "mapred.number.of.records";
|
||||
|
||||
/* SSL keystore resource */
|
||||
public static final String CONF_LABEL_SSL_KEYSTORE = "dfs.https.client.keystore.resource";
|
||||
|
||||
/* If input is based -f <<source listing>>, file containing the src paths */
|
||||
public static final String CONF_LABEL_LISTING_FILE_PATH = "distcp.listing.file.path";
|
||||
|
||||
/* Directory where the mapreduce job will write to. If not atomic commit, then same
|
||||
as CONF_LABEL_TARGET_FINAL_PATH
|
||||
*/
|
||||
public static final String CONF_LABEL_TARGET_WORK_PATH = "distcp.target.work.path";
|
||||
|
||||
/* Directory where the final data will be committed to. If not atomic commit, then same
|
||||
as CONF_LABEL_TARGET_WORK_PATH
|
||||
*/
|
||||
public static final String CONF_LABEL_TARGET_FINAL_PATH = "distcp.target.final.path";
|
||||
|
||||
/**
|
||||
* DistCp job id for consumers of the Disctp
|
||||
*/
|
||||
public static final String CONF_LABEL_DISTCP_JOB_ID = "distcp.job.id";
|
||||
|
||||
/* Meta folder where the job's intermediate data is kept */
|
||||
public static final String CONF_LABEL_META_FOLDER = "distcp.meta.folder";
|
||||
|
||||
/**
|
||||
* Conf label for SSL Trust-store location.
|
||||
*/
|
||||
public static final String CONF_LABEL_SSL_TRUST_STORE_LOCATION
|
||||
= "ssl.client.truststore.location";
|
||||
|
||||
/**
|
||||
* Conf label for SSL Key-store location.
|
||||
*/
|
||||
public static final String CONF_LABEL_SSL_KEY_STORE_LOCATION
|
||||
= "ssl.client.keystore.location";
|
||||
|
||||
/**
|
||||
* Constants for DistCp return code to shell / consumer of ToolRunner's run
|
||||
*/
|
||||
public static final int SUCCESS = 0;
|
||||
public static final int INVALID_ARGUMENT = -1;
|
||||
public static final int DUPLICATE_INPUT = -2;
|
||||
public static final int UNKNOWN_ERROR = -999;
|
||||
}
|
|
@ -0,0 +1,218 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.commons.cli.Option;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* Enumeration mapping configuration keys to distcp command line
|
||||
* options.
|
||||
*/
|
||||
public enum DistCpOptionSwitch {
|
||||
|
||||
/**
|
||||
* Ignores any failures during copy, and continues with rest.
|
||||
* Logs failures in a file
|
||||
*/
|
||||
IGNORE_FAILURES(DistCpConstants.CONF_LABEL_IGNORE_FAILURES,
|
||||
new Option("i", false, "Ignore failures during copy")),
|
||||
|
||||
/**
|
||||
* Preserves status of file/path in the target.
|
||||
* Default behavior with -p, is to preserve replication,
|
||||
* block size, user, group and permission on the target file
|
||||
*
|
||||
* If any of the optional switches are present among rbugp, then
|
||||
* only the corresponding file attribute is preserved
|
||||
*
|
||||
*/
|
||||
PRESERVE_STATUS(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
||||
new Option("p", true, "preserve status (rbugp)" +
|
||||
"(replication, block-size, user, group, permission)")),
|
||||
|
||||
/**
|
||||
* Update target location by copying only files that are missing
|
||||
* in the target. This can be used to periodically sync two folders
|
||||
* across source and target. Typically used with DELETE_MISSING
|
||||
* Incompatible with ATOMIC_COMMIT
|
||||
*/
|
||||
SYNC_FOLDERS(DistCpConstants.CONF_LABEL_SYNC_FOLDERS,
|
||||
new Option("update", false, "Update target, copying only missing" +
|
||||
"files or directories")),
|
||||
|
||||
/**
|
||||
* Deletes missing files in target that are missing from source
|
||||
* This allows the target to be in sync with the source contents
|
||||
* Typically used in conjunction with SYNC_FOLDERS
|
||||
* Incompatible with ATOMIC_COMMIT
|
||||
*/
|
||||
DELETE_MISSING(DistCpConstants.CONF_LABEL_DELETE_MISSING,
|
||||
new Option("delete", false, "Delete from target, " +
|
||||
"files missing in source")),
|
||||
|
||||
/**
|
||||
* Configuration file to use with hftps:// for securely copying
|
||||
* files across clusters. Typically the configuration file contains
|
||||
* truststore/keystore information such as location, password and type
|
||||
*/
|
||||
SSL_CONF(DistCpConstants.CONF_LABEL_SSL_CONF,
|
||||
new Option("mapredSslConf", true, "Configuration for ssl config file" +
|
||||
", to use with hftps://")),
|
||||
|
||||
/**
|
||||
* Max number of maps to use during copy. DistCp will split work
|
||||
* as equally as possible among these maps
|
||||
*/
|
||||
MAX_MAPS(DistCpConstants.CONF_LABEL_MAX_MAPS,
|
||||
new Option("m", true, "Max number of concurrent maps to use for copy")),
|
||||
|
||||
/**
|
||||
* Source file listing can be provided to DistCp in a file.
|
||||
* This allows DistCp to copy random list of files from source
|
||||
* and copy them to target
|
||||
*/
|
||||
SOURCE_FILE_LISTING(DistCpConstants.CONF_LABEL_SOURCE_LISTING,
|
||||
new Option("f", true, "List of files that need to be copied")),
|
||||
|
||||
/**
|
||||
* Copy all the source files and commit them atomically to the target
|
||||
* This is typically useful in cases where there is a process
|
||||
* polling for availability of a file/dir. This option is incompatible
|
||||
* with SYNC_FOLDERS & DELETE_MISSING
|
||||
*/
|
||||
ATOMIC_COMMIT(DistCpConstants.CONF_LABEL_ATOMIC_COPY,
|
||||
new Option("atomic", false, "Commit all changes or none")),
|
||||
|
||||
/**
|
||||
* Work path to be used only in conjunction in Atomic commit
|
||||
*/
|
||||
WORK_PATH(DistCpConstants.CONF_LABEL_WORK_PATH,
|
||||
new Option("tmp", true, "Intermediate work path to be used for atomic commit")),
|
||||
|
||||
/**
|
||||
* Log path where distcp output logs are written to
|
||||
*/
|
||||
LOG_PATH(DistCpConstants.CONF_LABEL_LOG_PATH,
|
||||
new Option("log", true, "Folder on DFS where distcp execution logs are saved")),
|
||||
|
||||
/**
|
||||
* Copy strategy is use. This could be dynamic or uniform size etc.
|
||||
* DistCp would use an appropriate input format based on this.
|
||||
*/
|
||||
COPY_STRATEGY(DistCpConstants.CONF_LABEL_COPY_STRATEGY,
|
||||
new Option("strategy", true, "Copy strategy to use. Default is " +
|
||||
"dividing work based on file sizes")),
|
||||
|
||||
/**
|
||||
* Skip CRC checks between source and target, when determining what
|
||||
* files need to be copied.
|
||||
*/
|
||||
SKIP_CRC(DistCpConstants.CONF_LABEL_SKIP_CRC,
|
||||
new Option("skipcrccheck", false, "Whether to skip CRC checks between " +
|
||||
"source and target paths.")),
|
||||
|
||||
/**
|
||||
* Overwrite target-files unconditionally.
|
||||
*/
|
||||
OVERWRITE(DistCpConstants.CONF_LABEL_OVERWRITE,
|
||||
new Option("overwrite", false, "Choose to overwrite target files " +
|
||||
"unconditionally, even if they exist.")),
|
||||
|
||||
/**
|
||||
* Should DisctpExecution be blocking
|
||||
*/
|
||||
BLOCKING("",
|
||||
new Option("async", false, "Should distcp execution be blocking")),
|
||||
|
||||
FILE_LIMIT("",
|
||||
new Option("filelimit", true, "(Deprecated!) Limit number of files " +
|
||||
"copied to <= n")),
|
||||
|
||||
SIZE_LIMIT("",
|
||||
new Option("sizelimit", true, "(Deprecated!) Limit number of files " +
|
||||
"copied to <= n bytes")),
|
||||
|
||||
/**
|
||||
* Specify bandwidth per map in MB
|
||||
*/
|
||||
BANDWIDTH(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
|
||||
new Option("bandwidth", true, "Specify bandwidth per map in MB"));
|
||||
|
||||
private final String confLabel;
|
||||
private final Option option;
|
||||
|
||||
DistCpOptionSwitch(String confLabel, Option option) {
|
||||
this.confLabel = confLabel;
|
||||
this.option = option;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Configuration label for the option
|
||||
* @return configuration label name
|
||||
*/
|
||||
public String getConfigLabel() {
|
||||
return confLabel;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get CLI Option corresponding to the distcp option
|
||||
* @return option
|
||||
*/
|
||||
public Option getOption() {
|
||||
return option;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Switch symbol
|
||||
* @return switch symbol char
|
||||
*/
|
||||
public String getSwitch() {
|
||||
return option.getOpt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return super.name() + " {" +
|
||||
"confLabel='" + confLabel + '\'' +
|
||||
", option=" + option + '}';
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper function to add an option to hadoop configuration object
|
||||
* @param conf - Configuration object to include the option
|
||||
* @param option - Option to add
|
||||
* @param value - Value
|
||||
*/
|
||||
public static void addToConf(Configuration conf,
|
||||
DistCpOptionSwitch option,
|
||||
String value) {
|
||||
conf.set(option.getConfigLabel(), value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper function to set an option to hadoop configuration object
|
||||
* @param conf - Configuration object to include the option
|
||||
* @param option - Option to add
|
||||
*/
|
||||
public static void addToConf(Configuration conf,
|
||||
DistCpOptionSwitch option) {
|
||||
conf.set(option.getConfigLabel(), "true");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,525 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
|
||||
import java.util.EnumSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
/**
|
||||
* The Options class encapsulates all DistCp options.
|
||||
* These may be set from command-line (via the OptionsParser)
|
||||
* or may be set manually.
|
||||
*/
|
||||
public class DistCpOptions {
|
||||
|
||||
private boolean atomicCommit = false;
|
||||
private boolean syncFolder = false;
|
||||
private boolean deleteMissing = false;
|
||||
private boolean ignoreFailures = false;
|
||||
private boolean overwrite = false;
|
||||
private boolean skipCRC = false;
|
||||
private boolean blocking = true;
|
||||
|
||||
private int maxMaps = DistCpConstants.DEFAULT_MAPS;
|
||||
private int mapBandwidth = DistCpConstants.DEFAULT_BANDWIDTH_MB;
|
||||
|
||||
private String sslConfigurationFile;
|
||||
|
||||
private String copyStrategy = DistCpConstants.UNIFORMSIZE;
|
||||
|
||||
private EnumSet<FileAttribute> preserveStatus = EnumSet.noneOf(FileAttribute.class);
|
||||
|
||||
private Path atomicWorkPath;
|
||||
|
||||
private Path logPath;
|
||||
|
||||
private Path sourceFileListing;
|
||||
private List<Path> sourcePaths;
|
||||
|
||||
private Path targetPath;
|
||||
|
||||
public static enum FileAttribute{
|
||||
REPLICATION, BLOCKSIZE, USER, GROUP, PERMISSION;
|
||||
|
||||
public static FileAttribute getAttribute(char symbol) {
|
||||
for (FileAttribute attribute : values()) {
|
||||
if (attribute.name().charAt(0) == Character.toUpperCase(symbol)) {
|
||||
return attribute;
|
||||
}
|
||||
}
|
||||
throw new NoSuchElementException("No attribute for " + symbol);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor, to initialize source/target paths.
|
||||
* @param sourcePaths List of source-paths (including wildcards)
|
||||
* to be copied to target.
|
||||
* @param targetPath Destination path for the dist-copy.
|
||||
*/
|
||||
public DistCpOptions(List<Path> sourcePaths, Path targetPath) {
|
||||
assert sourcePaths != null && !sourcePaths.isEmpty() : "Invalid source paths";
|
||||
assert targetPath != null : "Invalid Target path";
|
||||
|
||||
this.sourcePaths = sourcePaths;
|
||||
this.targetPath = targetPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor, to initialize source/target paths.
|
||||
* @param sourceFileListing File containing list of source paths
|
||||
* @param targetPath Destination path for the dist-copy.
|
||||
*/
|
||||
public DistCpOptions(Path sourceFileListing, Path targetPath) {
|
||||
assert sourceFileListing != null : "Invalid source paths";
|
||||
assert targetPath != null : "Invalid Target path";
|
||||
|
||||
this.sourceFileListing = sourceFileListing;
|
||||
this.targetPath = targetPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy constructor.
|
||||
* @param that DistCpOptions being copied from.
|
||||
*/
|
||||
public DistCpOptions(DistCpOptions that) {
|
||||
if (this != that && that != null) {
|
||||
this.atomicCommit = that.atomicCommit;
|
||||
this.syncFolder = that.syncFolder;
|
||||
this.deleteMissing = that.deleteMissing;
|
||||
this.ignoreFailures = that.ignoreFailures;
|
||||
this.overwrite = that.overwrite;
|
||||
this.skipCRC = that.skipCRC;
|
||||
this.blocking = that.blocking;
|
||||
this.maxMaps = that.maxMaps;
|
||||
this.mapBandwidth = that.mapBandwidth;
|
||||
this.sslConfigurationFile = that.getSslConfigurationFile();
|
||||
this.copyStrategy = that.copyStrategy;
|
||||
this.preserveStatus = that.preserveStatus;
|
||||
this.atomicWorkPath = that.getAtomicWorkPath();
|
||||
this.logPath = that.getLogPath();
|
||||
this.sourceFileListing = that.getSourceFileListing();
|
||||
this.sourcePaths = that.getSourcePaths();
|
||||
this.targetPath = that.getTargetPath();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Should the data be committed atomically?
|
||||
*
|
||||
* @return true if data should be committed automically. false otherwise
|
||||
*/
|
||||
public boolean shouldAtomicCommit() {
|
||||
return atomicCommit;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set if data need to be committed automatically
|
||||
*
|
||||
* @param atomicCommit - boolean switch
|
||||
*/
|
||||
public void setAtomicCommit(boolean atomicCommit) {
|
||||
validate(DistCpOptionSwitch.ATOMIC_COMMIT, atomicCommit);
|
||||
this.atomicCommit = atomicCommit;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should the data be sync'ed between source and target paths?
|
||||
*
|
||||
* @return true if data should be sync'ed up. false otherwise
|
||||
*/
|
||||
public boolean shouldSyncFolder() {
|
||||
return syncFolder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set if source and target folder contents be sync'ed up
|
||||
*
|
||||
* @param syncFolder - boolean switch
|
||||
*/
|
||||
public void setSyncFolder(boolean syncFolder) {
|
||||
validate(DistCpOptionSwitch.SYNC_FOLDERS, syncFolder);
|
||||
this.syncFolder = syncFolder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should target files missing in source should be deleted?
|
||||
*
|
||||
* @return true if zoombie target files to be removed. false otherwise
|
||||
*/
|
||||
public boolean shouldDeleteMissing() {
|
||||
return deleteMissing;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set if files only present in target should be deleted
|
||||
*
|
||||
* @param deleteMissing - boolean switch
|
||||
*/
|
||||
public void setDeleteMissing(boolean deleteMissing) {
|
||||
validate(DistCpOptionSwitch.DELETE_MISSING, deleteMissing);
|
||||
this.deleteMissing = deleteMissing;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should failures be logged and ignored during copy?
|
||||
*
|
||||
* @return true if failures are to be logged and ignored. false otherwise
|
||||
*/
|
||||
public boolean shouldIgnoreFailures() {
|
||||
return ignoreFailures;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set if failures during copy be ignored
|
||||
*
|
||||
* @param ignoreFailures - boolean switch
|
||||
*/
|
||||
public void setIgnoreFailures(boolean ignoreFailures) {
|
||||
this.ignoreFailures = ignoreFailures;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should DistCp be running in blocking mode
|
||||
*
|
||||
* @return true if should run in blocking, false otherwise
|
||||
*/
|
||||
public boolean shouldBlock() {
|
||||
return blocking;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set if Disctp should run blocking or non-blocking
|
||||
*
|
||||
* @param blocking - boolean switch
|
||||
*/
|
||||
public void setBlocking(boolean blocking) {
|
||||
this.blocking = blocking;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should files be overwritten always?
|
||||
*
|
||||
* @return true if files in target that may exist before distcp, should always
|
||||
* be overwritten. false otherwise
|
||||
*/
|
||||
public boolean shouldOverwrite() {
|
||||
return overwrite;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set if files should always be overwritten on target
|
||||
*
|
||||
* @param overwrite - boolean switch
|
||||
*/
|
||||
public void setOverwrite(boolean overwrite) {
|
||||
validate(DistCpOptionSwitch.OVERWRITE, overwrite);
|
||||
this.overwrite = overwrite;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should CRC/checksum check be skipped while checking files are identical
|
||||
*
|
||||
* @return true if checksum check should be skipped while checking files are
|
||||
* identical. false otherwise
|
||||
*/
|
||||
public boolean shouldSkipCRC() {
|
||||
return skipCRC;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set if checksum comparison should be skipped while determining if
|
||||
* source and destination files are identical
|
||||
*
|
||||
* @param skipCRC - boolean switch
|
||||
*/
|
||||
public void setSkipCRC(boolean skipCRC) {
|
||||
validate(DistCpOptionSwitch.SKIP_CRC, skipCRC);
|
||||
this.skipCRC = skipCRC;
|
||||
}
|
||||
|
||||
/** Get the max number of maps to use for this copy
|
||||
*
|
||||
* @return Max number of maps
|
||||
*/
|
||||
public int getMaxMaps() {
|
||||
return maxMaps;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the max number of maps to use for copy
|
||||
*
|
||||
* @param maxMaps - Number of maps
|
||||
*/
|
||||
public void setMaxMaps(int maxMaps) {
|
||||
this.maxMaps = maxMaps;
|
||||
}
|
||||
|
||||
/** Get the map bandwidth in MB
|
||||
*
|
||||
* @return Bandwidth in MB
|
||||
*/
|
||||
public int getMapBandwidth() {
|
||||
return mapBandwidth;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set per map bandwidth
|
||||
*
|
||||
* @param mapBandwidth - per map bandwidth
|
||||
*/
|
||||
public void setMapBandwidth(int mapBandwidth) {
|
||||
assert mapBandwidth > 0 : "Bandwidth " + mapBandwidth + " is invalid (should be > 0)";
|
||||
this.mapBandwidth = mapBandwidth;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get path where the ssl configuration file is present to use for hftps://
|
||||
*
|
||||
* @return Path on local file system
|
||||
*/
|
||||
public String getSslConfigurationFile() {
|
||||
return sslConfigurationFile;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the SSL configuration file path to use with hftps:// (local path)
|
||||
*
|
||||
* @param sslConfigurationFile - Local ssl config file path
|
||||
*/
|
||||
public void setSslConfigurationFile(String sslConfigurationFile) {
|
||||
this.sslConfigurationFile = sslConfigurationFile;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an iterator with the list of file attributes to preserve
|
||||
*
|
||||
* @return iterator of file attributes to preserve
|
||||
*/
|
||||
public Iterator<FileAttribute> preserveAttributes() {
|
||||
return preserveStatus.iterator();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the input attibute should be preserved or not
|
||||
*
|
||||
* @param attribute - Attribute to check
|
||||
* @return True if attribute should be preserved, false otherwise
|
||||
*/
|
||||
public boolean shouldPreserve(FileAttribute attribute) {
|
||||
return preserveStatus.contains(attribute);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add file attributes that need to be preserved. This method may be
|
||||
* called multiple times to add attributes.
|
||||
*
|
||||
* @param fileAttribute - Attribute to add, one at a time
|
||||
*/
|
||||
public void preserve(FileAttribute fileAttribute) {
|
||||
for (FileAttribute attribute : preserveStatus) {
|
||||
if (attribute.equals(fileAttribute)) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
preserveStatus.add(fileAttribute);
|
||||
}
|
||||
|
||||
/** Get work path for atomic commit. If null, the work
|
||||
* path would be parentOf(targetPath) + "/._WIP_" + nameOf(targetPath)
|
||||
*
|
||||
* @return Atomic work path on the target cluster. Null if not set
|
||||
*/
|
||||
public Path getAtomicWorkPath() {
|
||||
return atomicWorkPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the work path for atomic commit
|
||||
*
|
||||
* @param atomicWorkPath - Path on the target cluster
|
||||
*/
|
||||
public void setAtomicWorkPath(Path atomicWorkPath) {
|
||||
this.atomicWorkPath = atomicWorkPath;
|
||||
}
|
||||
|
||||
/** Get output directory for writing distcp logs. Otherwise logs
|
||||
* are temporarily written to JobStagingDir/_logs and deleted
|
||||
* upon job completion
|
||||
*
|
||||
* @return Log output path on the cluster where distcp job is run
|
||||
*/
|
||||
public Path getLogPath() {
|
||||
return logPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the log path where distcp output logs are stored
|
||||
* Uses JobStagingDir/_logs by default
|
||||
*
|
||||
* @param logPath - Path where logs will be saved
|
||||
*/
|
||||
public void setLogPath(Path logPath) {
|
||||
this.logPath = logPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the copy strategy to use. Uses appropriate input format
|
||||
*
|
||||
* @return copy strategy to use
|
||||
*/
|
||||
public String getCopyStrategy() {
|
||||
return copyStrategy;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the copy strategy to use. Should map to a strategy implementation
|
||||
* in distp-default.xml
|
||||
*
|
||||
* @param copyStrategy - copy Strategy to use
|
||||
*/
|
||||
public void setCopyStrategy(String copyStrategy) {
|
||||
this.copyStrategy = copyStrategy;
|
||||
}
|
||||
|
||||
/**
|
||||
* File path (hdfs:// or file://) that contains the list of actual
|
||||
* files to copy
|
||||
*
|
||||
* @return - Source listing file path
|
||||
*/
|
||||
public Path getSourceFileListing() {
|
||||
return sourceFileListing;
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for sourcePaths.
|
||||
* @return List of source-paths.
|
||||
*/
|
||||
public List<Path> getSourcePaths() {
|
||||
return sourcePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Setter for sourcePaths.
|
||||
* @param sourcePaths The new list of source-paths.
|
||||
*/
|
||||
public void setSourcePaths(List<Path> sourcePaths) {
|
||||
assert sourcePaths != null && sourcePaths.size() != 0;
|
||||
this.sourcePaths = sourcePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for the targetPath.
|
||||
* @return The target-path.
|
||||
*/
|
||||
public Path getTargetPath() {
|
||||
return targetPath;
|
||||
}
|
||||
|
||||
public void validate(DistCpOptionSwitch option, boolean value) {
|
||||
|
||||
boolean syncFolder = (option == DistCpOptionSwitch.SYNC_FOLDERS ?
|
||||
value : this.syncFolder);
|
||||
boolean overwrite = (option == DistCpOptionSwitch.OVERWRITE ?
|
||||
value : this.overwrite);
|
||||
boolean deleteMissing = (option == DistCpOptionSwitch.DELETE_MISSING ?
|
||||
value : this.deleteMissing);
|
||||
boolean atomicCommit = (option == DistCpOptionSwitch.ATOMIC_COMMIT ?
|
||||
value : this.atomicCommit);
|
||||
boolean skipCRC = (option == DistCpOptionSwitch.SKIP_CRC ?
|
||||
value : this.skipCRC);
|
||||
|
||||
if (syncFolder && atomicCommit) {
|
||||
throw new IllegalArgumentException("Atomic commit can't be used with " +
|
||||
"sync folder or overwrite options");
|
||||
}
|
||||
|
||||
if (deleteMissing && !(overwrite || syncFolder)) {
|
||||
throw new IllegalArgumentException("Delete missing is applicable " +
|
||||
"only with update or overwrite options");
|
||||
}
|
||||
|
||||
if (overwrite && syncFolder) {
|
||||
throw new IllegalArgumentException("Overwrite and update options are " +
|
||||
"mutually exclusive");
|
||||
}
|
||||
|
||||
if (!syncFolder && skipCRC) {
|
||||
throw new IllegalArgumentException("Skip CRC is valid only with update options");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Add options to configuration. These will be used in the Mapper/committer
|
||||
*
|
||||
* @param conf - Configruation object to which the options need to be added
|
||||
*/
|
||||
public void appendToConf(Configuration conf) {
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.ATOMIC_COMMIT,
|
||||
String.valueOf(atomicCommit));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.IGNORE_FAILURES,
|
||||
String.valueOf(ignoreFailures));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.SYNC_FOLDERS,
|
||||
String.valueOf(syncFolder));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.DELETE_MISSING,
|
||||
String.valueOf(deleteMissing));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.OVERWRITE,
|
||||
String.valueOf(overwrite));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.SKIP_CRC,
|
||||
String.valueOf(skipCRC));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.BANDWIDTH,
|
||||
String.valueOf(mapBandwidth));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.PRESERVE_STATUS,
|
||||
DistCpUtils.packAttributes(preserveStatus));
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to easily string-ify Options, for logging.
|
||||
*
|
||||
* @return String representation of the Options.
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DistCpOptions{" +
|
||||
"atomicCommit=" + atomicCommit +
|
||||
", syncFolder=" + syncFolder +
|
||||
", deleteMissing=" + deleteMissing +
|
||||
", ignoreFailures=" + ignoreFailures +
|
||||
", maxMaps=" + maxMaps +
|
||||
", sslConfigurationFile='" + sslConfigurationFile + '\'' +
|
||||
", copyStrategy='" + copyStrategy + '\'' +
|
||||
", sourceFileListing=" + sourceFileListing +
|
||||
", sourcePaths=" + sourcePaths +
|
||||
", targetPath=" + targetPath +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DistCpOptions clone() throws CloneNotSupportedException {
|
||||
return (DistCpOptions) super.clone();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* FileBasedCopyListing implements the CopyListing interface,
|
||||
* to create the copy-listing for DistCp,
|
||||
* by iterating over all source paths mentioned in a specified input-file.
|
||||
*/
|
||||
public class FileBasedCopyListing extends CopyListing {
|
||||
|
||||
private final CopyListing globbedListing;
|
||||
/**
|
||||
* Constructor, to initialize base-class.
|
||||
* @param configuration The input Configuration object.
|
||||
* @param credentials - Credentials object on which the FS delegation tokens are cached. If null
|
||||
* delegation token caching is skipped
|
||||
*/
|
||||
public FileBasedCopyListing(Configuration configuration, Credentials credentials) {
|
||||
super(configuration, credentials);
|
||||
globbedListing = new GlobbedCopyListing(getConf(), credentials);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected void validatePaths(DistCpOptions options)
|
||||
throws IOException, InvalidInputException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of CopyListing::buildListing().
|
||||
* Iterates over all source paths mentioned in the input-file.
|
||||
* @param pathToListFile Path on HDFS where the listing file is written.
|
||||
* @param options Input Options for DistCp (indicating source/target paths.)
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void doBuildListing(Path pathToListFile, DistCpOptions options) throws IOException {
|
||||
DistCpOptions newOption = new DistCpOptions(options);
|
||||
newOption.setSourcePaths(fetchFileList(options.getSourceFileListing()));
|
||||
globbedListing.buildListing(pathToListFile, newOption);
|
||||
}
|
||||
|
||||
private List<Path> fetchFileList(Path sourceListing) throws IOException {
|
||||
List<Path> result = new ArrayList<Path>();
|
||||
FileSystem fs = sourceListing.getFileSystem(getConf());
|
||||
BufferedReader input = null;
|
||||
try {
|
||||
input = new BufferedReader(new InputStreamReader(fs.open(sourceListing)));
|
||||
String line = input.readLine();
|
||||
while (line != null) {
|
||||
result.add(new Path(line));
|
||||
line = input.readLine();
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeStream(input);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected long getBytesToCopy() {
|
||||
return globbedListing.getBytesToCopy();
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected long getNumberOfPaths() {
|
||||
return globbedListing.getNumberOfPaths();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,105 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* GlobbedCopyListing implements the CopyListing interface, to create the copy
|
||||
* listing-file by "globbing" all specified source paths (wild-cards and all.)
|
||||
*/
|
||||
public class GlobbedCopyListing extends CopyListing {
|
||||
private static final Log LOG = LogFactory.getLog(GlobbedCopyListing.class);
|
||||
|
||||
private final CopyListing simpleListing;
|
||||
/**
|
||||
* Constructor, to initialize the configuration.
|
||||
* @param configuration The input Configuration object.
|
||||
* @param credentials Credentials object on which the FS delegation tokens are cached. If null
|
||||
* delegation token caching is skipped
|
||||
*/
|
||||
public GlobbedCopyListing(Configuration configuration, Credentials credentials) {
|
||||
super(configuration, credentials);
|
||||
simpleListing = new SimpleCopyListing(getConf(), credentials) ;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected void validatePaths(DistCpOptions options)
|
||||
throws IOException, InvalidInputException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of CopyListing::buildListing().
|
||||
* Creates the copy listing by "globbing" all source-paths.
|
||||
* @param pathToListingFile The location at which the copy-listing file
|
||||
* is to be created.
|
||||
* @param options Input Options for DistCp (indicating source/target paths.)
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void doBuildListing(Path pathToListingFile,
|
||||
DistCpOptions options) throws IOException {
|
||||
|
||||
List<Path> globbedPaths = new ArrayList<Path>();
|
||||
if (options.getSourcePaths().isEmpty()) {
|
||||
throw new InvalidInputException("Nothing to process. Source paths::EMPTY");
|
||||
}
|
||||
|
||||
for (Path p : options.getSourcePaths()) {
|
||||
FileSystem fs = p.getFileSystem(getConf());
|
||||
FileStatus[] inputs = fs.globStatus(p);
|
||||
|
||||
if(inputs != null && inputs.length > 0) {
|
||||
for (FileStatus onePath: inputs) {
|
||||
globbedPaths.add(onePath.getPath());
|
||||
}
|
||||
} else {
|
||||
throw new InvalidInputException(p + " doesn't exist");
|
||||
}
|
||||
}
|
||||
|
||||
DistCpOptions optionsGlobbed = new DistCpOptions(options);
|
||||
optionsGlobbed.setSourcePaths(globbedPaths);
|
||||
simpleListing.buildListing(pathToListingFile, optionsGlobbed);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected long getBytesToCopy() {
|
||||
return simpleListing.getBytesToCopy();
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected long getNumberOfPaths() {
|
||||
return simpleListing.getNumberOfPaths();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,246 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.commons.cli.*;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
/**
|
||||
* The OptionsParser parses out the command-line options passed to DistCp,
|
||||
* and interprets those specific to DistCp, to create an Options object.
|
||||
*/
|
||||
public class OptionsParser {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(OptionsParser.class);
|
||||
|
||||
private static final Options cliOptions = new Options();
|
||||
|
||||
static {
|
||||
for (DistCpOptionSwitch option : DistCpOptionSwitch.values()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Adding option " + option.getOption());
|
||||
}
|
||||
cliOptions.addOption(option.getOption());
|
||||
}
|
||||
}
|
||||
|
||||
private static class CustomParser extends GnuParser {
|
||||
@Override
|
||||
protected String[] flatten(Options options, String[] arguments, boolean stopAtNonOption) {
|
||||
for (int index = 0; index < arguments.length; index++) {
|
||||
if (arguments[index].equals("-" + DistCpOptionSwitch.PRESERVE_STATUS.getSwitch())) {
|
||||
arguments[index] = "-prbugp";
|
||||
}
|
||||
}
|
||||
return super.flatten(options, arguments, stopAtNonOption);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The parse method parses the command-line options, and creates
|
||||
* a corresponding Options object.
|
||||
* @param args Command-line arguments (excluding the options consumed
|
||||
* by the GenericOptionsParser).
|
||||
* @return The Options object, corresponding to the specified command-line.
|
||||
* @throws IllegalArgumentException: Thrown if the parse fails.
|
||||
*/
|
||||
public static DistCpOptions parse(String args[]) throws IllegalArgumentException {
|
||||
|
||||
CommandLineParser parser = new CustomParser();
|
||||
|
||||
CommandLine command;
|
||||
try {
|
||||
command = parser.parse(cliOptions, args, true);
|
||||
} catch (ParseException e) {
|
||||
throw new IllegalArgumentException("Unable to parse arguments. " +
|
||||
Arrays.toString(args), e);
|
||||
}
|
||||
|
||||
DistCpOptions option;
|
||||
Path targetPath;
|
||||
List<Path> sourcePaths = new ArrayList<Path>();
|
||||
|
||||
String leftOverArgs[] = command.getArgs();
|
||||
if (leftOverArgs == null || leftOverArgs.length < 1) {
|
||||
throw new IllegalArgumentException("Target path not specified");
|
||||
}
|
||||
|
||||
//Last Argument is the target path
|
||||
targetPath = new Path(leftOverArgs[leftOverArgs.length -1].trim());
|
||||
|
||||
//Copy any source paths in the arguments to the list
|
||||
for (int index = 0; index < leftOverArgs.length - 1; index++) {
|
||||
sourcePaths.add(new Path(leftOverArgs[index].trim()));
|
||||
}
|
||||
|
||||
/* If command has source file listing, use it else, fall back on source paths in args
|
||||
If both are present, throw exception and bail */
|
||||
if (command.hasOption(DistCpOptionSwitch.SOURCE_FILE_LISTING.getSwitch())) {
|
||||
if (!sourcePaths.isEmpty()) {
|
||||
throw new IllegalArgumentException("Both source file listing and source paths present");
|
||||
}
|
||||
option = new DistCpOptions(new Path(getVal(command, DistCpOptionSwitch.
|
||||
SOURCE_FILE_LISTING.getSwitch())), targetPath);
|
||||
} else {
|
||||
if (sourcePaths.isEmpty()) {
|
||||
throw new IllegalArgumentException("Neither source file listing nor source paths present");
|
||||
}
|
||||
option = new DistCpOptions(sourcePaths, targetPath);
|
||||
}
|
||||
|
||||
//Process all the other option switches and set options appropriately
|
||||
if (command.hasOption(DistCpOptionSwitch.IGNORE_FAILURES.getSwitch())) {
|
||||
option.setIgnoreFailures(true);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.ATOMIC_COMMIT.getSwitch())) {
|
||||
option.setAtomicCommit(true);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.WORK_PATH.getSwitch()) &&
|
||||
option.shouldAtomicCommit()) {
|
||||
String workPath = getVal(command, DistCpOptionSwitch.WORK_PATH.getSwitch());
|
||||
if (workPath != null && !workPath.isEmpty()) {
|
||||
option.setAtomicWorkPath(new Path(workPath));
|
||||
}
|
||||
} else if (command.hasOption(DistCpOptionSwitch.WORK_PATH.getSwitch())) {
|
||||
throw new IllegalArgumentException("-tmp work-path can only be specified along with -atomic");
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.LOG_PATH.getSwitch())) {
|
||||
option.setLogPath(new Path(getVal(command, DistCpOptionSwitch.LOG_PATH.getSwitch())));
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.SYNC_FOLDERS.getSwitch())) {
|
||||
option.setSyncFolder(true);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.OVERWRITE.getSwitch())) {
|
||||
option.setOverwrite(true);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.DELETE_MISSING.getSwitch())) {
|
||||
option.setDeleteMissing(true);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.SKIP_CRC.getSwitch())) {
|
||||
option.setSkipCRC(true);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.BLOCKING.getSwitch())) {
|
||||
option.setBlocking(false);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.BANDWIDTH.getSwitch())) {
|
||||
try {
|
||||
Integer mapBandwidth = Integer.parseInt(
|
||||
getVal(command, DistCpOptionSwitch.BANDWIDTH.getSwitch()).trim());
|
||||
option.setMapBandwidth(mapBandwidth);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("Bandwidth specified is invalid: " +
|
||||
getVal(command, DistCpOptionSwitch.BANDWIDTH.getSwitch()), e);
|
||||
}
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.SSL_CONF.getSwitch())) {
|
||||
option.setSslConfigurationFile(command.
|
||||
getOptionValue(DistCpOptionSwitch.SSL_CONF.getSwitch()));
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.MAX_MAPS.getSwitch())) {
|
||||
try {
|
||||
Integer maps = Integer.parseInt(
|
||||
getVal(command, DistCpOptionSwitch.MAX_MAPS.getSwitch()).trim());
|
||||
option.setMaxMaps(maps);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("Number of maps is invalid: " +
|
||||
getVal(command, DistCpOptionSwitch.MAX_MAPS.getSwitch()), e);
|
||||
}
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.COPY_STRATEGY.getSwitch())) {
|
||||
option.setCopyStrategy(
|
||||
getVal(command, DistCpOptionSwitch.COPY_STRATEGY.getSwitch()));
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.PRESERVE_STATUS.getSwitch())) {
|
||||
String attributes =
|
||||
getVal(command, DistCpOptionSwitch.PRESERVE_STATUS.getSwitch());
|
||||
if (attributes == null || attributes.isEmpty()) {
|
||||
for (FileAttribute attribute : FileAttribute.values()) {
|
||||
option.preserve(attribute);
|
||||
}
|
||||
} else {
|
||||
for (int index = 0; index < attributes.length(); index++) {
|
||||
option.preserve(FileAttribute.
|
||||
getAttribute(attributes.charAt(index)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.FILE_LIMIT.getSwitch())) {
|
||||
String fileLimitString = getVal(command,
|
||||
DistCpOptionSwitch.FILE_LIMIT.getSwitch().trim());
|
||||
try {
|
||||
Integer.parseInt(fileLimitString);
|
||||
}
|
||||
catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("File-limit is invalid: "
|
||||
+ fileLimitString, e);
|
||||
}
|
||||
LOG.warn(DistCpOptionSwitch.FILE_LIMIT.getSwitch() + " is a deprecated" +
|
||||
" option. Ignoring.");
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.SIZE_LIMIT.getSwitch())) {
|
||||
String sizeLimitString = getVal(command,
|
||||
DistCpOptionSwitch.SIZE_LIMIT.getSwitch().trim());
|
||||
try {
|
||||
Long.parseLong(sizeLimitString);
|
||||
}
|
||||
catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("Size-limit is invalid: "
|
||||
+ sizeLimitString, e);
|
||||
}
|
||||
LOG.warn(DistCpOptionSwitch.SIZE_LIMIT.getSwitch() + " is a deprecated" +
|
||||
" option. Ignoring.");
|
||||
}
|
||||
|
||||
return option;
|
||||
}
|
||||
|
||||
private static String getVal(CommandLine command, String swtch) {
|
||||
String optionValue = command.getOptionValue(swtch);
|
||||
if (optionValue == null) {
|
||||
return null;
|
||||
} else {
|
||||
return optionValue.trim();
|
||||
}
|
||||
}
|
||||
|
||||
public static void usage() {
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
formatter.printHelp("distcp OPTIONS [source_path...] <target_path>\n\nOPTIONS", cliOptions);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,275 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.mapreduce.security.TokenCache;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.Stack;
|
||||
|
||||
/**
|
||||
* The SimpleCopyListing is responsible for making the exhaustive list of
|
||||
* all files/directories under its specified list of input-paths.
|
||||
* These are written into the specified copy-listing file.
|
||||
* Note: The SimpleCopyListing doesn't handle wild-cards in the input-paths.
|
||||
*/
|
||||
public class SimpleCopyListing extends CopyListing {
|
||||
private static final Log LOG = LogFactory.getLog(SimpleCopyListing.class);
|
||||
|
||||
private long totalPaths = 0;
|
||||
private long totalBytesToCopy = 0;
|
||||
|
||||
/**
|
||||
* Protected constructor, to initialize configuration.
|
||||
*
|
||||
* @param configuration The input configuration, with which the source/target FileSystems may be accessed.
|
||||
* @param credentials - Credentials object on which the FS delegation tokens are cached. If null
|
||||
* delegation token caching is skipped
|
||||
*/
|
||||
protected SimpleCopyListing(Configuration configuration, Credentials credentials) {
|
||||
super(configuration, credentials);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void validatePaths(DistCpOptions options)
|
||||
throws IOException, InvalidInputException {
|
||||
|
||||
Path targetPath = options.getTargetPath();
|
||||
FileSystem targetFS = targetPath.getFileSystem(getConf());
|
||||
boolean targetIsFile = targetFS.isFile(targetPath);
|
||||
|
||||
//If target is a file, then source has to be single file
|
||||
if (targetIsFile) {
|
||||
if (options.getSourcePaths().size() > 1) {
|
||||
throw new InvalidInputException("Multiple source being copied to a file: " +
|
||||
targetPath);
|
||||
}
|
||||
|
||||
Path srcPath = options.getSourcePaths().get(0);
|
||||
FileSystem sourceFS = srcPath.getFileSystem(getConf());
|
||||
if (!sourceFS.isFile(srcPath)) {
|
||||
throw new InvalidInputException("Cannot copy " + srcPath +
|
||||
", which is not a file to " + targetPath);
|
||||
}
|
||||
}
|
||||
|
||||
if (options.shouldAtomicCommit() && targetFS.exists(targetPath)) {
|
||||
throw new InvalidInputException("Target path for atomic-commit already exists: " +
|
||||
targetPath + ". Cannot atomic-commit to pre-existing target-path.");
|
||||
}
|
||||
|
||||
for (Path path: options.getSourcePaths()) {
|
||||
FileSystem fs = path.getFileSystem(getConf());
|
||||
if (!fs.exists(path)) {
|
||||
throw new InvalidInputException(path + " doesn't exist");
|
||||
}
|
||||
}
|
||||
|
||||
/* This is requires to allow map tasks to access each of the source
|
||||
clusters. This would retrieve the delegation token for each unique
|
||||
file system and add them to job's private credential store
|
||||
*/
|
||||
Credentials credentials = getCredentials();
|
||||
if (credentials != null) {
|
||||
Path[] inputPaths = options.getSourcePaths().toArray(new Path[1]);
|
||||
TokenCache.obtainTokensForNamenodes(credentials, inputPaths, getConf());
|
||||
}
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public void doBuildListing(Path pathToListingFile, DistCpOptions options) throws IOException {
|
||||
|
||||
SequenceFile.Writer fileListWriter = null;
|
||||
|
||||
try {
|
||||
fileListWriter = getWriter(pathToListingFile);
|
||||
|
||||
for (Path path: options.getSourcePaths()) {
|
||||
FileSystem sourceFS = path.getFileSystem(getConf());
|
||||
path = makeQualified(path);
|
||||
|
||||
FileStatus rootStatus = sourceFS.getFileStatus(path);
|
||||
Path sourcePathRoot = computeSourceRootPath(rootStatus, options);
|
||||
boolean localFile = (rootStatus.getClass() != FileStatus.class);
|
||||
|
||||
FileStatus[] sourceFiles = sourceFS.listStatus(path);
|
||||
if (sourceFiles != null && sourceFiles.length > 0) {
|
||||
for (FileStatus sourceStatus: sourceFiles) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Recording source-path: " + sourceStatus.getPath() + " for copy.");
|
||||
}
|
||||
writeToFileListing(fileListWriter, sourceStatus, sourcePathRoot, localFile);
|
||||
|
||||
if (isDirectoryAndNotEmpty(sourceFS, sourceStatus)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Traversing non-empty source dir: " + sourceStatus.getPath());
|
||||
}
|
||||
traverseNonEmptyDirectory(fileListWriter, sourceStatus, sourcePathRoot, localFile);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
writeToFileListing(fileListWriter, rootStatus, sourcePathRoot, localFile);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeStream(fileListWriter);
|
||||
}
|
||||
}
|
||||
|
||||
private Path computeSourceRootPath(FileStatus sourceStatus,
|
||||
DistCpOptions options) throws IOException {
|
||||
|
||||
Path target = options.getTargetPath();
|
||||
FileSystem targetFS = target.getFileSystem(getConf());
|
||||
|
||||
boolean solitaryFile = options.getSourcePaths().size() == 1
|
||||
&& !sourceStatus.isDirectory();
|
||||
|
||||
if (solitaryFile) {
|
||||
if (targetFS.isFile(target) || !targetFS.exists(target)) {
|
||||
return sourceStatus.getPath();
|
||||
} else {
|
||||
return sourceStatus.getPath().getParent();
|
||||
}
|
||||
} else {
|
||||
boolean specialHandling = (options.getSourcePaths().size() == 1 && !targetFS.exists(target)) ||
|
||||
options.shouldSyncFolder() || options.shouldOverwrite();
|
||||
|
||||
return specialHandling && sourceStatus.isDirectory() ? sourceStatus.getPath() :
|
||||
sourceStatus.getPath().getParent();
|
||||
}
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected long getBytesToCopy() {
|
||||
return totalBytesToCopy;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected long getNumberOfPaths() {
|
||||
return totalPaths;
|
||||
}
|
||||
|
||||
private Path makeQualified(Path path) throws IOException {
|
||||
final FileSystem fs = path.getFileSystem(getConf());
|
||||
return path.makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
}
|
||||
|
||||
private SequenceFile.Writer getWriter(Path pathToListFile) throws IOException {
|
||||
FileSystem fs = pathToListFile.getFileSystem(getConf());
|
||||
if (fs.exists(pathToListFile)) {
|
||||
fs.delete(pathToListFile, false);
|
||||
}
|
||||
return SequenceFile.createWriter(getConf(),
|
||||
SequenceFile.Writer.file(pathToListFile),
|
||||
SequenceFile.Writer.keyClass(Text.class),
|
||||
SequenceFile.Writer.valueClass(FileStatus.class),
|
||||
SequenceFile.Writer.compression(SequenceFile.CompressionType.NONE));
|
||||
}
|
||||
|
||||
private static boolean isDirectoryAndNotEmpty(FileSystem fileSystem,
|
||||
FileStatus fileStatus) throws IOException {
|
||||
return fileStatus.isDirectory() && getChildren(fileSystem, fileStatus).length > 0;
|
||||
}
|
||||
|
||||
private static FileStatus[] getChildren(FileSystem fileSystem,
|
||||
FileStatus parent) throws IOException {
|
||||
return fileSystem.listStatus(parent.getPath());
|
||||
}
|
||||
|
||||
private void traverseNonEmptyDirectory(SequenceFile.Writer fileListWriter,
|
||||
FileStatus sourceStatus,
|
||||
Path sourcePathRoot, boolean localFile)
|
||||
throws IOException {
|
||||
FileSystem sourceFS = sourcePathRoot.getFileSystem(getConf());
|
||||
Stack<FileStatus> pathStack = new Stack<FileStatus>();
|
||||
pathStack.push(sourceStatus);
|
||||
|
||||
while (!pathStack.isEmpty()) {
|
||||
for (FileStatus child: getChildren(sourceFS, pathStack.pop())) {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug("Recording source-path: "
|
||||
+ sourceStatus.getPath() + " for copy.");
|
||||
writeToFileListing(fileListWriter, child, sourcePathRoot, localFile);
|
||||
if (isDirectoryAndNotEmpty(sourceFS, child)) {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug("Traversing non-empty source dir: "
|
||||
+ sourceStatus.getPath());
|
||||
pathStack.push(child);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void writeToFileListing(SequenceFile.Writer fileListWriter,
|
||||
FileStatus fileStatus, Path sourcePathRoot,
|
||||
boolean localFile) throws IOException {
|
||||
if (fileStatus.getPath().equals(sourcePathRoot) && fileStatus.isDirectory())
|
||||
return; // Skip the root-paths.
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("REL PATH: " + DistCpUtils.getRelativePath(sourcePathRoot,
|
||||
fileStatus.getPath()) + ", FULL PATH: " + fileStatus.getPath());
|
||||
}
|
||||
|
||||
FileStatus status = fileStatus;
|
||||
if (localFile) {
|
||||
status = getFileStatus(fileStatus);
|
||||
}
|
||||
|
||||
fileListWriter.append(new Text(DistCpUtils.getRelativePath(sourcePathRoot,
|
||||
fileStatus.getPath())), status);
|
||||
fileListWriter.sync();
|
||||
|
||||
if (!fileStatus.isDirectory()) {
|
||||
totalBytesToCopy += fileStatus.getLen();
|
||||
}
|
||||
totalPaths++;
|
||||
}
|
||||
|
||||
private static final ByteArrayOutputStream buffer = new ByteArrayOutputStream(64);
|
||||
private DataInputBuffer in = new DataInputBuffer();
|
||||
|
||||
private FileStatus getFileStatus(FileStatus fileStatus) throws IOException {
|
||||
FileStatus status = new FileStatus();
|
||||
|
||||
buffer.reset();
|
||||
DataOutputStream out = new DataOutputStream(buffer);
|
||||
fileStatus.write(out);
|
||||
|
||||
in.reset(buffer.toByteArray(), 0, buffer.size());
|
||||
status.readFields(in);
|
||||
return status;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,297 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
|
||||
import org.apache.hadoop.tools.*;
|
||||
import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* The CopyCommitter class is DistCp's OutputCommitter implementation. It is
|
||||
* responsible for handling the completion/cleanup of the DistCp run.
|
||||
* Specifically, it does the following:
|
||||
* 1. Cleanup of the meta-folder (where DistCp maintains its file-list, etc.)
|
||||
* 2. Preservation of user/group/replication-factor on any directories that
|
||||
* have been copied. (Files are taken care of in their map-tasks.)
|
||||
* 3. Atomic-move of data from the temporary work-folder to the final path
|
||||
* (if atomic-commit was opted for).
|
||||
* 4. Deletion of files from the target that are missing at source (if opted for).
|
||||
* 5. Cleanup of any partially copied files, from previous, failed attempts.
|
||||
*/
|
||||
public class CopyCommitter extends FileOutputCommitter {
|
||||
private static final Log LOG = LogFactory.getLog(CopyCommitter.class);
|
||||
|
||||
private final TaskAttemptContext taskAttemptContext;
|
||||
|
||||
/**
|
||||
* Create a output committer
|
||||
*
|
||||
* @param outputPath the job's output path
|
||||
* @param context the task's context
|
||||
* @throws IOException - Exception if any
|
||||
*/
|
||||
public CopyCommitter(Path outputPath, TaskAttemptContext context) throws IOException {
|
||||
super(outputPath, context);
|
||||
this.taskAttemptContext = context;
|
||||
}
|
||||
|
||||
/** @inheritDoc */
|
||||
@Override
|
||||
public void commitJob(JobContext jobContext) throws IOException {
|
||||
Configuration conf = jobContext.getConfiguration();
|
||||
super.commitJob(jobContext);
|
||||
|
||||
cleanupTempFiles(jobContext);
|
||||
|
||||
String attributes = conf.get(DistCpConstants.CONF_LABEL_PRESERVE_STATUS);
|
||||
if (attributes != null && !attributes.isEmpty()) {
|
||||
preserveFileAttributesForDirectories(conf);
|
||||
}
|
||||
|
||||
try {
|
||||
if (conf.getBoolean(DistCpConstants.CONF_LABEL_DELETE_MISSING, false)) {
|
||||
deleteMissing(conf);
|
||||
} else if (conf.getBoolean(DistCpConstants.CONF_LABEL_ATOMIC_COPY, false)) {
|
||||
commitData(conf);
|
||||
}
|
||||
taskAttemptContext.setStatus("Commit Successful");
|
||||
}
|
||||
finally {
|
||||
cleanup(conf);
|
||||
}
|
||||
}
|
||||
|
||||
/** @inheritDoc */
|
||||
@Override
|
||||
public void abortJob(JobContext jobContext,
|
||||
JobStatus.State state) throws IOException {
|
||||
try {
|
||||
super.abortJob(jobContext, state);
|
||||
} finally {
|
||||
cleanupTempFiles(jobContext);
|
||||
cleanup(jobContext.getConfiguration());
|
||||
}
|
||||
}
|
||||
|
||||
private void cleanupTempFiles(JobContext context) {
|
||||
try {
|
||||
Configuration conf = context.getConfiguration();
|
||||
|
||||
Path targetWorkPath = new Path(conf.get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH));
|
||||
FileSystem targetFS = targetWorkPath.getFileSystem(conf);
|
||||
|
||||
String jobId = context.getJobID().toString();
|
||||
deleteAttemptTempFiles(targetWorkPath, targetFS, jobId);
|
||||
deleteAttemptTempFiles(targetWorkPath.getParent(), targetFS, jobId);
|
||||
} catch (Throwable t) {
|
||||
LOG.warn("Unable to cleanup temp files", t);
|
||||
}
|
||||
}
|
||||
|
||||
private void deleteAttemptTempFiles(Path targetWorkPath,
|
||||
FileSystem targetFS,
|
||||
String jobId) throws IOException {
|
||||
|
||||
FileStatus[] tempFiles = targetFS.globStatus(
|
||||
new Path(targetWorkPath, ".distcp.tmp." + jobId.replaceAll("job","attempt") + "*"));
|
||||
|
||||
if (tempFiles != null && tempFiles.length > 0) {
|
||||
for (FileStatus file : tempFiles) {
|
||||
LOG.info("Cleaning up " + file.getPath());
|
||||
targetFS.delete(file.getPath(), false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanup meta folder and other temporary files
|
||||
*
|
||||
* @param conf - Job Configuration
|
||||
*/
|
||||
private void cleanup(Configuration conf) {
|
||||
Path metaFolder = new Path(conf.get(DistCpConstants.CONF_LABEL_META_FOLDER));
|
||||
try {
|
||||
FileSystem fs = metaFolder.getFileSystem(conf);
|
||||
LOG.info("Cleaning up temporary work folder: " + metaFolder);
|
||||
fs.delete(metaFolder, true);
|
||||
} catch (IOException ignore) {
|
||||
LOG.error("Exception encountered ", ignore);
|
||||
}
|
||||
}
|
||||
|
||||
// This method changes the target-directories' file-attributes (owner,
|
||||
// user/group permissions, etc.) based on the corresponding source directories.
|
||||
private void preserveFileAttributesForDirectories(Configuration conf) throws IOException {
|
||||
String attrSymbols = conf.get(DistCpConstants.CONF_LABEL_PRESERVE_STATUS);
|
||||
LOG.info("About to preserve attributes: " + attrSymbols);
|
||||
|
||||
EnumSet<FileAttribute> attributes = DistCpUtils.unpackAttributes(attrSymbols);
|
||||
|
||||
Path sourceListing = new Path(conf.get(DistCpConstants.CONF_LABEL_LISTING_FILE_PATH));
|
||||
FileSystem clusterFS = sourceListing.getFileSystem(conf);
|
||||
SequenceFile.Reader sourceReader = new SequenceFile.Reader(conf,
|
||||
SequenceFile.Reader.file(sourceListing));
|
||||
long totalLen = clusterFS.getFileStatus(sourceListing).getLen();
|
||||
|
||||
Path targetRoot = new Path(conf.get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH));
|
||||
|
||||
long preservedEntries = 0;
|
||||
try {
|
||||
FileStatus srcFileStatus = new FileStatus();
|
||||
Text srcRelPath = new Text();
|
||||
|
||||
// Iterate over every source path that was copied.
|
||||
while (sourceReader.next(srcRelPath, srcFileStatus)) {
|
||||
// File-attributes for files are set at the time of copy,
|
||||
// in the map-task.
|
||||
if (! srcFileStatus.isDirectory()) continue;
|
||||
|
||||
Path targetFile = new Path(targetRoot.toString() + "/" + srcRelPath);
|
||||
|
||||
// Skip the root folder.
|
||||
// Status can't be preserved on root-folder. (E.g. multiple paths may
|
||||
// be copied to a single target folder. Which source-attributes to use
|
||||
// on the target is undefined.)
|
||||
if (targetRoot.equals(targetFile)) continue;
|
||||
|
||||
FileSystem targetFS = targetFile.getFileSystem(conf);
|
||||
DistCpUtils.preserve(targetFS, targetFile, srcFileStatus, attributes);
|
||||
|
||||
taskAttemptContext.progress();
|
||||
taskAttemptContext.setStatus("Preserving status on directory entries. [" +
|
||||
sourceReader.getPosition() * 100 / totalLen + "%]");
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeStream(sourceReader);
|
||||
}
|
||||
LOG.info("Preserved status on " + preservedEntries + " dir entries on target");
|
||||
}
|
||||
|
||||
// This method deletes "extra" files from the target, if they're not
|
||||
// available at the source.
|
||||
private void deleteMissing(Configuration conf) throws IOException {
|
||||
LOG.info("-delete option is enabled. About to remove entries from " +
|
||||
"target that are missing in source");
|
||||
|
||||
// Sort the source-file listing alphabetically.
|
||||
Path sourceListing = new Path(conf.get(DistCpConstants.CONF_LABEL_LISTING_FILE_PATH));
|
||||
FileSystem clusterFS = sourceListing.getFileSystem(conf);
|
||||
Path sortedSourceListing = DistCpUtils.sortListing(clusterFS, conf, sourceListing);
|
||||
|
||||
// Similarly, create the listing of target-files. Sort alphabetically.
|
||||
Path targetListing = new Path(sourceListing.getParent(), "targetListing.seq");
|
||||
CopyListing target = new GlobbedCopyListing(new Configuration(conf), null);
|
||||
|
||||
List<Path> targets = new ArrayList<Path>(1);
|
||||
Path targetFinalPath = new Path(conf.get(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH));
|
||||
targets.add(targetFinalPath);
|
||||
DistCpOptions options = new DistCpOptions(targets, new Path("/NONE"));
|
||||
|
||||
target.buildListing(targetListing, options);
|
||||
Path sortedTargetListing = DistCpUtils.sortListing(clusterFS, conf, targetListing);
|
||||
long totalLen = clusterFS.getFileStatus(sortedTargetListing).getLen();
|
||||
|
||||
SequenceFile.Reader sourceReader = new SequenceFile.Reader(conf,
|
||||
SequenceFile.Reader.file(sortedSourceListing));
|
||||
SequenceFile.Reader targetReader = new SequenceFile.Reader(conf,
|
||||
SequenceFile.Reader.file(sortedTargetListing));
|
||||
|
||||
// Walk both source and target file listings.
|
||||
// Delete all from target that doesn't also exist on source.
|
||||
long deletedEntries = 0;
|
||||
try {
|
||||
FileStatus srcFileStatus = new FileStatus();
|
||||
Text srcRelPath = new Text();
|
||||
FileStatus trgtFileStatus = new FileStatus();
|
||||
Text trgtRelPath = new Text();
|
||||
|
||||
FileSystem targetFS = targetFinalPath.getFileSystem(conf);
|
||||
boolean srcAvailable = sourceReader.next(srcRelPath, srcFileStatus);
|
||||
while (targetReader.next(trgtRelPath, trgtFileStatus)) {
|
||||
// Skip sources that don't exist on target.
|
||||
while (srcAvailable && trgtRelPath.compareTo(srcRelPath) > 0) {
|
||||
srcAvailable = sourceReader.next(srcRelPath, srcFileStatus);
|
||||
}
|
||||
|
||||
if (srcAvailable && trgtRelPath.equals(srcRelPath)) continue;
|
||||
|
||||
// Target doesn't exist at source. Delete.
|
||||
boolean result = (!targetFS.exists(trgtFileStatus.getPath()) ||
|
||||
targetFS.delete(trgtFileStatus.getPath(), true));
|
||||
if (result) {
|
||||
LOG.info("Deleted " + trgtFileStatus.getPath() + " - Missing at source");
|
||||
deletedEntries++;
|
||||
} else {
|
||||
throw new IOException("Unable to delete " + trgtFileStatus.getPath());
|
||||
}
|
||||
taskAttemptContext.progress();
|
||||
taskAttemptContext.setStatus("Deleting missing files from target. [" +
|
||||
targetReader.getPosition() * 100 / totalLen + "%]");
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeStream(sourceReader);
|
||||
IOUtils.closeStream(targetReader);
|
||||
}
|
||||
LOG.info("Deleted " + deletedEntries + " from target: " + targets.get(0));
|
||||
}
|
||||
|
||||
private void commitData(Configuration conf) throws IOException {
|
||||
|
||||
Path workDir = new Path(conf.get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH));
|
||||
Path finalDir = new Path(conf.get(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH));
|
||||
FileSystem targetFS = workDir.getFileSystem(conf);
|
||||
|
||||
LOG.info("Atomic commit enabled. Moving " + workDir + " to " + finalDir);
|
||||
if (targetFS.exists(finalDir) && targetFS.exists(workDir)) {
|
||||
LOG.error("Pre-existing final-path found at: " + finalDir);
|
||||
throw new IOException("Target-path can't be committed to because it " +
|
||||
"exists at " + finalDir + ". Copied data is in temp-dir: " + workDir + ". ");
|
||||
}
|
||||
|
||||
boolean result = targetFS.rename(workDir, finalDir);
|
||||
if (!result) {
|
||||
LOG.warn("Rename failed. Perhaps data already moved. Verifying...");
|
||||
result = targetFS.exists(finalDir) && !targetFS.exists(workDir);
|
||||
}
|
||||
if (result) {
|
||||
LOG.info("Data committed successfully to " + finalDir);
|
||||
taskAttemptContext.setStatus("Data committed successfully to " + finalDir);
|
||||
} else {
|
||||
LOG.error("Unable to commit data to " + finalDir);
|
||||
throw new IOException("Atomic commit failed. Temporary data in " + workDir +
|
||||
", Unable to move to " + finalDir);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,330 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hadoop.mapreduce.Mapper;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.tools.DistCpOptionSwitch;
|
||||
import org.apache.hadoop.tools.DistCpOptions;
|
||||
import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.EnumSet;
|
||||
import java.util.Arrays;
|
||||
|
||||
/**
|
||||
* Mapper class that executes the DistCp copy operation.
|
||||
* Implements the o.a.h.mapreduce.Mapper<> interface.
|
||||
*/
|
||||
public class CopyMapper extends Mapper<Text, FileStatus, Text, Text> {
|
||||
|
||||
/**
|
||||
* Hadoop counters for the DistCp CopyMapper.
|
||||
* (These have been kept identical to the old DistCp,
|
||||
* for backward compatibility.)
|
||||
*/
|
||||
public static enum Counter {
|
||||
COPY, // Number of files received by the mapper for copy.
|
||||
SKIP, // Number of files skipped.
|
||||
FAIL, // Number of files that failed to be copied.
|
||||
BYTESCOPIED, // Number of bytes actually copied by the copy-mapper, total.
|
||||
BYTESEXPECTED,// Number of bytes expected to be copied.
|
||||
BYTESFAILED, // Number of bytes that failed to be copied.
|
||||
BYTESSKIPPED, // Number of bytes that were skipped from copy.
|
||||
}
|
||||
|
||||
private static Log LOG = LogFactory.getLog(CopyMapper.class);
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
private boolean syncFolders = false;
|
||||
private boolean ignoreFailures = false;
|
||||
private boolean skipCrc = false;
|
||||
private boolean overWrite = false;
|
||||
private EnumSet<FileAttribute> preserve = EnumSet.noneOf(FileAttribute.class);
|
||||
|
||||
private FileSystem targetFS = null;
|
||||
private Path targetWorkPath = null;
|
||||
|
||||
/**
|
||||
* Implementation of the Mapper::setup() method. This extracts the DistCp-
|
||||
* options specified in the Job's configuration, to set up the Job.
|
||||
* @param context Mapper's context.
|
||||
* @throws IOException On IO failure.
|
||||
* @throws InterruptedException If the job is interrupted.
|
||||
*/
|
||||
@Override
|
||||
public void setup(Context context) throws IOException, InterruptedException {
|
||||
conf = context.getConfiguration();
|
||||
|
||||
syncFolders = conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false);
|
||||
ignoreFailures = conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false);
|
||||
skipCrc = conf.getBoolean(DistCpOptionSwitch.SKIP_CRC.getConfigLabel(), false);
|
||||
overWrite = conf.getBoolean(DistCpOptionSwitch.OVERWRITE.getConfigLabel(), false);
|
||||
preserve = DistCpUtils.unpackAttributes(conf.get(DistCpOptionSwitch.
|
||||
PRESERVE_STATUS.getConfigLabel()));
|
||||
|
||||
targetWorkPath = new Path(conf.get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH));
|
||||
Path targetFinalPath = new Path(conf.get(
|
||||
DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH));
|
||||
targetFS = targetFinalPath.getFileSystem(conf);
|
||||
|
||||
if (targetFS.exists(targetFinalPath) && targetFS.isFile(targetFinalPath)) {
|
||||
overWrite = true; // When target is an existing file, overwrite it.
|
||||
}
|
||||
|
||||
if (conf.get(DistCpConstants.CONF_LABEL_SSL_CONF) != null) {
|
||||
initializeSSLConf(context);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize SSL Config if same is set in conf
|
||||
*
|
||||
* @throws IOException - If any
|
||||
*/
|
||||
private void initializeSSLConf(Context context) throws IOException {
|
||||
LOG.info("Initializing SSL configuration");
|
||||
|
||||
String workDir = conf.get(JobContext.JOB_LOCAL_DIR) + "/work";
|
||||
Path[] cacheFiles = context.getLocalCacheFiles();
|
||||
|
||||
Configuration sslConfig = new Configuration(false);
|
||||
String sslConfFileName = conf.get(DistCpConstants.CONF_LABEL_SSL_CONF);
|
||||
Path sslClient = findCacheFile(cacheFiles, sslConfFileName);
|
||||
if (sslClient == null) {
|
||||
LOG.warn("SSL Client config file not found. Was looking for " + sslConfFileName +
|
||||
" in " + Arrays.toString(cacheFiles));
|
||||
return;
|
||||
}
|
||||
sslConfig.addResource(sslClient);
|
||||
|
||||
String trustStoreFile = conf.get("ssl.client.truststore.location");
|
||||
Path trustStorePath = findCacheFile(cacheFiles, trustStoreFile);
|
||||
sslConfig.set("ssl.client.truststore.location", trustStorePath.toString());
|
||||
|
||||
String keyStoreFile = conf.get("ssl.client.keystore.location");
|
||||
Path keyStorePath = findCacheFile(cacheFiles, keyStoreFile);
|
||||
sslConfig.set("ssl.client.keystore.location", keyStorePath.toString());
|
||||
|
||||
try {
|
||||
OutputStream out = new FileOutputStream(workDir + "/" + sslConfFileName);
|
||||
try {
|
||||
sslConfig.writeXml(out);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
conf.set(DistCpConstants.CONF_LABEL_SSL_KEYSTORE, sslConfFileName);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to write out the ssl configuration. " +
|
||||
"Will fall back to default ssl-client.xml in class path, if there is one", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Find entry from distributed cache
|
||||
*
|
||||
* @param cacheFiles - All localized cache files
|
||||
* @param fileName - fileName to search
|
||||
* @return Path of the filename if found, else null
|
||||
*/
|
||||
private Path findCacheFile(Path[] cacheFiles, String fileName) {
|
||||
if (cacheFiles != null && cacheFiles.length > 0) {
|
||||
for (Path file : cacheFiles) {
|
||||
if (file.getName().equals(fileName)) {
|
||||
return file;
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of the Mapper<>::map(). Does the copy.
|
||||
* @param relPath The target path.
|
||||
* @param sourceFileStatus The source path.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void map(Text relPath, FileStatus sourceFileStatus, Context context)
|
||||
throws IOException, InterruptedException {
|
||||
Path sourcePath = sourceFileStatus.getPath();
|
||||
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug("DistCpMapper::map(): Received " + sourcePath + ", " + relPath);
|
||||
|
||||
Path target = new Path(targetWorkPath.makeQualified(targetFS.getUri(),
|
||||
targetFS.getWorkingDirectory()) + relPath.toString());
|
||||
|
||||
EnumSet<DistCpOptions.FileAttribute> fileAttributes
|
||||
= getFileAttributeSettings(context);
|
||||
|
||||
final String description = "Copying " + sourcePath + " to " + target;
|
||||
context.setStatus(description);
|
||||
|
||||
LOG.info(description);
|
||||
|
||||
try {
|
||||
FileStatus sourceCurrStatus;
|
||||
FileSystem sourceFS;
|
||||
try {
|
||||
sourceFS = sourcePath.getFileSystem(conf);
|
||||
sourceCurrStatus = sourceFS.getFileStatus(sourcePath);
|
||||
} catch (FileNotFoundException e) {
|
||||
throw new IOException(new RetriableFileCopyCommand.CopyReadException(e));
|
||||
}
|
||||
|
||||
FileStatus targetStatus = null;
|
||||
|
||||
try {
|
||||
targetStatus = targetFS.getFileStatus(target);
|
||||
} catch (FileNotFoundException ignore) {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug("Path could not be found: " + target, ignore);
|
||||
}
|
||||
|
||||
if (targetStatus != null && (targetStatus.isDirectory() != sourceCurrStatus.isDirectory())) {
|
||||
throw new IOException("Can't replace " + target + ". Target is " +
|
||||
getFileType(targetStatus) + ", Source is " + getFileType(sourceCurrStatus));
|
||||
}
|
||||
|
||||
if (sourceCurrStatus.isDirectory()) {
|
||||
createTargetDirsWithRetry(description, target, context);
|
||||
return;
|
||||
}
|
||||
|
||||
if (skipFile(sourceFS, sourceCurrStatus, target)) {
|
||||
LOG.info("Skipping copy of " + sourceCurrStatus.getPath()
|
||||
+ " to " + target);
|
||||
updateSkipCounters(context, sourceCurrStatus);
|
||||
context.write(null, new Text("SKIP: " + sourceCurrStatus.getPath()));
|
||||
}
|
||||
else {
|
||||
copyFileWithRetry(description, sourceCurrStatus, target, context,
|
||||
fileAttributes);
|
||||
}
|
||||
|
||||
DistCpUtils.preserve(target.getFileSystem(conf), target,
|
||||
sourceCurrStatus, fileAttributes);
|
||||
|
||||
} catch (IOException exception) {
|
||||
handleFailures(exception, sourceFileStatus, target, context);
|
||||
}
|
||||
}
|
||||
|
||||
private String getFileType(FileStatus fileStatus) {
|
||||
return fileStatus == null ? "N/A" : (fileStatus.isDirectory() ? "dir" : "file");
|
||||
}
|
||||
|
||||
private static EnumSet<DistCpOptions.FileAttribute>
|
||||
getFileAttributeSettings(Mapper.Context context) {
|
||||
String attributeString = context.getConfiguration().get(
|
||||
DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel());
|
||||
return DistCpUtils.unpackAttributes(attributeString);
|
||||
}
|
||||
|
||||
private void copyFileWithRetry(String description, FileStatus sourceFileStatus,
|
||||
Path target, Context context,
|
||||
EnumSet<DistCpOptions.FileAttribute> fileAttributes) throws IOException {
|
||||
|
||||
long bytesCopied;
|
||||
try {
|
||||
bytesCopied = (Long)new RetriableFileCopyCommand(description)
|
||||
.execute(sourceFileStatus, target, context, fileAttributes);
|
||||
} catch (Exception e) {
|
||||
context.setStatus("Copy Failure: " + sourceFileStatus.getPath());
|
||||
throw new IOException("File copy failed: " + sourceFileStatus.getPath() +
|
||||
" --> " + target, e);
|
||||
}
|
||||
incrementCounter(context, Counter.BYTESEXPECTED, sourceFileStatus.getLen());
|
||||
incrementCounter(context, Counter.BYTESCOPIED, bytesCopied);
|
||||
incrementCounter(context, Counter.COPY, 1);
|
||||
}
|
||||
|
||||
private void createTargetDirsWithRetry(String description,
|
||||
Path target, Context context) throws IOException {
|
||||
try {
|
||||
new RetriableDirectoryCreateCommand(description).execute(target, context);
|
||||
} catch (Exception e) {
|
||||
throw new IOException("mkdir failed for " + target, e);
|
||||
}
|
||||
incrementCounter(context, Counter.COPY, 1);
|
||||
}
|
||||
|
||||
private static void updateSkipCounters(Context context,
|
||||
FileStatus sourceFile) {
|
||||
incrementCounter(context, Counter.SKIP, 1);
|
||||
incrementCounter(context, Counter.BYTESSKIPPED, sourceFile.getLen());
|
||||
|
||||
}
|
||||
|
||||
private void handleFailures(IOException exception,
|
||||
FileStatus sourceFileStatus, Path target,
|
||||
Context context) throws IOException, InterruptedException {
|
||||
LOG.error("Failure in copying " + sourceFileStatus.getPath() + " to " +
|
||||
target, exception);
|
||||
|
||||
if (ignoreFailures && exception.getCause() instanceof
|
||||
RetriableFileCopyCommand.CopyReadException) {
|
||||
incrementCounter(context, Counter.FAIL, 1);
|
||||
incrementCounter(context, Counter.BYTESFAILED, sourceFileStatus.getLen());
|
||||
context.write(null, new Text("FAIL: " + sourceFileStatus.getPath() + " - " +
|
||||
StringUtils.stringifyException(exception)));
|
||||
}
|
||||
else
|
||||
throw exception;
|
||||
}
|
||||
|
||||
private static void incrementCounter(Context context, Counter counter,
|
||||
long value) {
|
||||
context.getCounter(counter).increment(value);
|
||||
}
|
||||
|
||||
private boolean skipFile(FileSystem sourceFS, FileStatus source, Path target)
|
||||
throws IOException {
|
||||
return targetFS.exists(target)
|
||||
&& !overWrite
|
||||
&& !mustUpdate(sourceFS, source, target);
|
||||
}
|
||||
|
||||
private boolean mustUpdate(FileSystem sourceFS, FileStatus source, Path target)
|
||||
throws IOException {
|
||||
final FileStatus targetFileStatus = targetFS.getFileStatus(target);
|
||||
|
||||
return syncFolders
|
||||
&& (
|
||||
targetFileStatus.getLen() != source.getLen()
|
||||
|| (!skipCrc &&
|
||||
!DistCpUtils.checksumsAreEqual(sourceFS,
|
||||
source.getPath(), targetFS, target))
|
||||
|| (source.getBlockSize() != targetFileStatus.getBlockSize() &&
|
||||
preserve.contains(FileAttribute.BLOCKSIZE))
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,124 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
|
||||
import org.apache.hadoop.mapreduce.security.TokenCache;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* The CopyOutputFormat is the Hadoop OutputFormat used in DistCp.
|
||||
* It sets up the Job's Configuration (in the Job-Context) with the settings
|
||||
* for the work-directory, final commit-directory, etc. It also sets the right
|
||||
* output-committer.
|
||||
* @param <K>
|
||||
* @param <V>
|
||||
*/
|
||||
public class CopyOutputFormat<K, V> extends TextOutputFormat<K, V> {
|
||||
|
||||
/**
|
||||
* Setter for the working directory for DistCp (where files will be copied
|
||||
* before they are moved to the final commit-directory.)
|
||||
* @param job The Job on whose configuration the working-directory is to be set.
|
||||
* @param workingDirectory The path to use as the working directory.
|
||||
*/
|
||||
public static void setWorkingDirectory(Job job, Path workingDirectory) {
|
||||
job.getConfiguration().set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH,
|
||||
workingDirectory.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Setter for the final directory for DistCp (where files copied will be
|
||||
* moved, atomically.)
|
||||
* @param job The Job on whose configuration the working-directory is to be set.
|
||||
* @param commitDirectory The path to use for final commit.
|
||||
*/
|
||||
public static void setCommitDirectory(Job job, Path commitDirectory) {
|
||||
job.getConfiguration().set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH,
|
||||
commitDirectory.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for the working directory.
|
||||
* @param job The Job from whose configuration the working-directory is to
|
||||
* be retrieved.
|
||||
* @return The working-directory Path.
|
||||
*/
|
||||
public static Path getWorkingDirectory(Job job) {
|
||||
return getWorkingDirectory(job.getConfiguration());
|
||||
}
|
||||
|
||||
private static Path getWorkingDirectory(Configuration conf) {
|
||||
String workingDirectory = conf.get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH);
|
||||
if (workingDirectory == null || workingDirectory.isEmpty()) {
|
||||
return null;
|
||||
} else {
|
||||
return new Path(workingDirectory);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for the final commit-directory.
|
||||
* @param job The Job from whose configuration the commit-directory is to be
|
||||
* retrieved.
|
||||
* @return The commit-directory Path.
|
||||
*/
|
||||
public static Path getCommitDirectory(Job job) {
|
||||
return getCommitDirectory(job.getConfiguration());
|
||||
}
|
||||
|
||||
private static Path getCommitDirectory(Configuration conf) {
|
||||
String commitDirectory = conf.get(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH);
|
||||
if (commitDirectory == null || commitDirectory.isEmpty()) {
|
||||
return null;
|
||||
} else {
|
||||
return new Path(commitDirectory);
|
||||
}
|
||||
}
|
||||
|
||||
/** @inheritDoc */
|
||||
@Override
|
||||
public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException {
|
||||
return new CopyCommitter(getOutputPath(context), context);
|
||||
}
|
||||
|
||||
/** @inheritDoc */
|
||||
@Override
|
||||
public void checkOutputSpecs(JobContext context) throws IOException {
|
||||
Configuration conf = context.getConfiguration();
|
||||
|
||||
if (getCommitDirectory(conf) == null) {
|
||||
throw new IllegalStateException("Commit directory not configured");
|
||||
}
|
||||
|
||||
Path workingPath = getWorkingDirectory(conf);
|
||||
if (workingPath == null) {
|
||||
throw new IllegalStateException("Working directory not configured");
|
||||
}
|
||||
|
||||
// get delegation token for outDir's file system
|
||||
TokenCache.obtainTokensForNamenodes(context.getCredentials(),
|
||||
new Path[] {workingPath}, conf);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.hadoop.tools.util.RetriableCommand;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.mapreduce.Mapper;
|
||||
|
||||
/**
|
||||
* This class extends Retriable command to implement the creation of directories
|
||||
* with retries on failure.
|
||||
*/
|
||||
public class RetriableDirectoryCreateCommand extends RetriableCommand {
|
||||
|
||||
/**
|
||||
* Constructor, taking a description of the action.
|
||||
* @param description Verbose description of the copy operation.
|
||||
*/
|
||||
public RetriableDirectoryCreateCommand(String description) {
|
||||
super(description);
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of RetriableCommand::doExecute().
|
||||
* This implements the actual mkdirs() functionality.
|
||||
* @param arguments Argument-list to the command.
|
||||
* @return Boolean. True, if the directory could be created successfully.
|
||||
* @throws Exception IOException, on failure to create the directory.
|
||||
*/
|
||||
@Override
|
||||
protected Object doExecute(Object... arguments) throws Exception {
|
||||
assert arguments.length == 2 : "Unexpected argument list.";
|
||||
Path target = (Path)arguments[0];
|
||||
Mapper.Context context = (Mapper.Context)arguments[1];
|
||||
|
||||
FileSystem targetFS = target.getFileSystem(context.getConfiguration());
|
||||
return targetFS.mkdirs(target);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,245 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.hadoop.tools.util.RetriableCommand;
|
||||
import org.apache.hadoop.tools.util.ThrottledInputStream;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.tools.DistCpOptions.*;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.hadoop.mapreduce.Mapper;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.EnumSet;
|
||||
|
||||
/**
|
||||
* This class extends RetriableCommand to implement the copy of files,
|
||||
* with retries on failure.
|
||||
*/
|
||||
public class RetriableFileCopyCommand extends RetriableCommand {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(RetriableFileCopyCommand.class);
|
||||
private static int BUFFER_SIZE = 8 * 1024;
|
||||
|
||||
/**
|
||||
* Constructor, taking a description of the action.
|
||||
* @param description Verbose description of the copy operation.
|
||||
*/
|
||||
public RetriableFileCopyCommand(String description) {
|
||||
super(description);
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of RetriableCommand::doExecute().
|
||||
* This is the actual copy-implementation.
|
||||
* @param arguments Argument-list to the command.
|
||||
* @return Number of bytes copied.
|
||||
* @throws Exception: CopyReadException, if there are read-failures. All other
|
||||
* failures are IOExceptions.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
protected Object doExecute(Object... arguments) throws Exception {
|
||||
assert arguments.length == 4 : "Unexpected argument list.";
|
||||
FileStatus source = (FileStatus)arguments[0];
|
||||
assert !source.isDirectory() : "Unexpected file-status. Expected file.";
|
||||
Path target = (Path)arguments[1];
|
||||
Mapper.Context context = (Mapper.Context)arguments[2];
|
||||
EnumSet<FileAttribute> fileAttributes
|
||||
= (EnumSet<FileAttribute>)arguments[3];
|
||||
return doCopy(source, target, context, fileAttributes);
|
||||
}
|
||||
|
||||
private long doCopy(FileStatus sourceFileStatus, Path target,
|
||||
Mapper.Context context,
|
||||
EnumSet<FileAttribute> fileAttributes)
|
||||
throws IOException {
|
||||
|
||||
Path tmpTargetPath = getTmpFile(target, context);
|
||||
final Configuration configuration = context.getConfiguration();
|
||||
FileSystem targetFS = target.getFileSystem(configuration);
|
||||
|
||||
try {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Copying " + sourceFileStatus.getPath() + " to " + target);
|
||||
LOG.debug("Tmp-file path: " + tmpTargetPath);
|
||||
}
|
||||
FileSystem sourceFS = sourceFileStatus.getPath().getFileSystem(
|
||||
configuration);
|
||||
long bytesRead = copyToTmpFile(tmpTargetPath, targetFS, sourceFileStatus,
|
||||
context, fileAttributes);
|
||||
|
||||
compareFileLengths(sourceFileStatus, tmpTargetPath, configuration, bytesRead);
|
||||
compareCheckSums(sourceFS, sourceFileStatus.getPath(), targetFS, tmpTargetPath);
|
||||
promoteTmpToTarget(tmpTargetPath, target, targetFS);
|
||||
return bytesRead;
|
||||
|
||||
} finally {
|
||||
if (targetFS.exists(tmpTargetPath))
|
||||
targetFS.delete(tmpTargetPath, false);
|
||||
}
|
||||
}
|
||||
|
||||
private long copyToTmpFile(Path tmpTargetPath, FileSystem targetFS,
|
||||
FileStatus sourceFileStatus, Mapper.Context context,
|
||||
EnumSet<FileAttribute> fileAttributes)
|
||||
throws IOException {
|
||||
OutputStream outStream = new BufferedOutputStream(targetFS.create(
|
||||
tmpTargetPath, true, BUFFER_SIZE,
|
||||
getReplicationFactor(fileAttributes, sourceFileStatus, targetFS),
|
||||
getBlockSize(fileAttributes, sourceFileStatus, targetFS), context));
|
||||
return copyBytes(sourceFileStatus, outStream, BUFFER_SIZE, true, context);
|
||||
}
|
||||
|
||||
private void compareFileLengths(FileStatus sourceFileStatus, Path target,
|
||||
Configuration configuration, long bytesRead)
|
||||
throws IOException {
|
||||
final Path sourcePath = sourceFileStatus.getPath();
|
||||
FileSystem fs = sourcePath.getFileSystem(configuration);
|
||||
if (fs.getFileStatus(sourcePath).getLen() != bytesRead)
|
||||
throw new IOException("Mismatch in length of source:" + sourcePath
|
||||
+ " and target:" + target);
|
||||
}
|
||||
|
||||
private void compareCheckSums(FileSystem sourceFS, Path source,
|
||||
FileSystem targetFS, Path target)
|
||||
throws IOException {
|
||||
if (!DistCpUtils.checksumsAreEqual(sourceFS, source, targetFS, target))
|
||||
throw new IOException("Check-sum mismatch between "
|
||||
+ source + " and " + target);
|
||||
|
||||
}
|
||||
|
||||
//If target file exists and unable to delete target - fail
|
||||
//If target doesn't exist and unable to create parent folder - fail
|
||||
//If target is successfully deleted and parent exists, if rename fails - fail
|
||||
private void promoteTmpToTarget(Path tmpTarget, Path target, FileSystem fs)
|
||||
throws IOException {
|
||||
if ((fs.exists(target) && !fs.delete(target, false))
|
||||
|| (!fs.exists(target.getParent()) && !fs.mkdirs(target.getParent()))
|
||||
|| !fs.rename(tmpTarget, target)) {
|
||||
throw new IOException("Failed to promote tmp-file:" + tmpTarget
|
||||
+ " to: " + target);
|
||||
}
|
||||
}
|
||||
|
||||
private Path getTmpFile(Path target, Mapper.Context context) {
|
||||
Path targetWorkPath = new Path(context.getConfiguration().
|
||||
get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH));
|
||||
|
||||
Path root = target.equals(targetWorkPath)? targetWorkPath.getParent() : targetWorkPath;
|
||||
LOG.info("Creating temp file: " +
|
||||
new Path(root, ".distcp.tmp." + context.getTaskAttemptID().toString()));
|
||||
return new Path(root, ".distcp.tmp." + context.getTaskAttemptID().toString());
|
||||
}
|
||||
|
||||
private long copyBytes(FileStatus sourceFileStatus, OutputStream outStream,
|
||||
int bufferSize, boolean mustCloseStream,
|
||||
Mapper.Context context) throws IOException {
|
||||
Path source = sourceFileStatus.getPath();
|
||||
byte buf[] = new byte[bufferSize];
|
||||
ThrottledInputStream inStream = null;
|
||||
long totalBytesRead = 0;
|
||||
|
||||
try {
|
||||
inStream = getInputStream(source, context.getConfiguration());
|
||||
int bytesRead = readBytes(inStream, buf);
|
||||
while (bytesRead >= 0) {
|
||||
totalBytesRead += bytesRead;
|
||||
outStream.write(buf, 0, bytesRead);
|
||||
updateContextStatus(totalBytesRead, context, sourceFileStatus);
|
||||
bytesRead = inStream.read(buf);
|
||||
}
|
||||
} finally {
|
||||
if (mustCloseStream)
|
||||
IOUtils.cleanup(LOG, outStream, inStream);
|
||||
}
|
||||
|
||||
return totalBytesRead;
|
||||
}
|
||||
|
||||
private void updateContextStatus(long totalBytesRead, Mapper.Context context,
|
||||
FileStatus sourceFileStatus) {
|
||||
StringBuilder message = new StringBuilder(DistCpUtils.getFormatter()
|
||||
.format(totalBytesRead * 100.0f / sourceFileStatus.getLen()));
|
||||
message.append("% ")
|
||||
.append(description).append(" [")
|
||||
.append(DistCpUtils.getStringDescriptionFor(totalBytesRead))
|
||||
.append('/')
|
||||
.append(DistCpUtils.getStringDescriptionFor(sourceFileStatus.getLen()))
|
||||
.append(']');
|
||||
context.setStatus(message.toString());
|
||||
}
|
||||
|
||||
private static int readBytes(InputStream inStream, byte buf[])
|
||||
throws IOException {
|
||||
try {
|
||||
return inStream.read(buf);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new CopyReadException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static ThrottledInputStream getInputStream(Path path, Configuration conf)
|
||||
throws IOException {
|
||||
try {
|
||||
FileSystem fs = path.getFileSystem(conf);
|
||||
long bandwidthMB = conf.getInt(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
|
||||
DistCpConstants.DEFAULT_BANDWIDTH_MB);
|
||||
return new ThrottledInputStream(new BufferedInputStream(fs.open(path)),
|
||||
bandwidthMB * 1024 * 1024);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new CopyReadException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static short getReplicationFactor(
|
||||
EnumSet<FileAttribute> fileAttributes,
|
||||
FileStatus sourceFile, FileSystem targetFS) {
|
||||
return fileAttributes.contains(FileAttribute.REPLICATION)?
|
||||
sourceFile.getReplication() : targetFS.getDefaultReplication();
|
||||
}
|
||||
|
||||
private static long getBlockSize(
|
||||
EnumSet<FileAttribute> fileAttributes,
|
||||
FileStatus sourceFile, FileSystem targetFS) {
|
||||
return fileAttributes.contains(FileAttribute.BLOCKSIZE)?
|
||||
sourceFile.getBlockSize() : targetFS.getDefaultBlockSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* Special subclass of IOException. This is used to distinguish read-operation
|
||||
* failures from other kinds of IOExceptions.
|
||||
* The failure to read from source is dealt with specially, in the CopyMapper.
|
||||
* Such failures may be skipped if the DistCpOptions indicate so.
|
||||
* Write failures are intolerable, and amount to CopyMapper failure.
|
||||
*/
|
||||
public static class CopyReadException extends IOException {
|
||||
public CopyReadException(Throwable rootCause) {
|
||||
super(rootCause);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,169 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader;
|
||||
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* UniformSizeInputFormat extends the InputFormat<> class, to produce
|
||||
* input-splits for DistCp.
|
||||
* It looks at the copy-listing and groups the contents into input-splits such
|
||||
* that the total-number of bytes to be copied for each input split is
|
||||
* uniform.
|
||||
*/
|
||||
public class UniformSizeInputFormat extends InputFormat<Text, FileStatus> {
|
||||
private static final Log LOG
|
||||
= LogFactory.getLog(UniformSizeInputFormat.class);
|
||||
|
||||
/**
|
||||
* Implementation of InputFormat::getSplits(). Returns a list of InputSplits,
|
||||
* such that the number of bytes to be copied for all the splits are
|
||||
* approximately equal.
|
||||
* @param context JobContext for the job.
|
||||
* @return The list of uniformly-distributed input-splits.
|
||||
* @throws IOException: On failure.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public List<InputSplit> getSplits(JobContext context)
|
||||
throws IOException, InterruptedException {
|
||||
Configuration configuration = context.getConfiguration();
|
||||
int numSplits = DistCpUtils.getInt(configuration,
|
||||
JobContext.NUM_MAPS);
|
||||
|
||||
if (numSplits == 0) return new ArrayList<InputSplit>();
|
||||
|
||||
return getSplits(configuration, numSplits,
|
||||
DistCpUtils.getLong(configuration,
|
||||
DistCpConstants.CONF_LABEL_TOTAL_BYTES_TO_BE_COPIED));
|
||||
}
|
||||
|
||||
private List<InputSplit> getSplits(Configuration configuration, int numSplits,
|
||||
long totalSizeBytes) throws IOException {
|
||||
List<InputSplit> splits = new ArrayList<InputSplit>(numSplits);
|
||||
long nBytesPerSplit = (long) Math.ceil(totalSizeBytes * 1.0 / numSplits);
|
||||
|
||||
FileStatus srcFileStatus = new FileStatus();
|
||||
Text srcRelPath = new Text();
|
||||
long currentSplitSize = 0;
|
||||
long lastSplitStart = 0;
|
||||
long lastPosition = 0;
|
||||
|
||||
final Path listingFilePath = getListingFilePath(configuration);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Average bytes per map: " + nBytesPerSplit +
|
||||
", Number of maps: " + numSplits + ", total size: " + totalSizeBytes);
|
||||
}
|
||||
SequenceFile.Reader reader=null;
|
||||
try {
|
||||
reader = getListingFileReader(configuration);
|
||||
while (reader.next(srcRelPath, srcFileStatus)) {
|
||||
// If adding the current file would cause the bytes per map to exceed
|
||||
// limit. Add the current file to new split
|
||||
if (currentSplitSize + srcFileStatus.getLen() > nBytesPerSplit && lastPosition != 0) {
|
||||
FileSplit split = new FileSplit(listingFilePath, lastSplitStart,
|
||||
lastPosition - lastSplitStart, null);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug ("Creating split : " + split + ", bytes in split: " + currentSplitSize);
|
||||
}
|
||||
splits.add(split);
|
||||
lastSplitStart = lastPosition;
|
||||
currentSplitSize = 0;
|
||||
}
|
||||
currentSplitSize += srcFileStatus.getLen();
|
||||
lastPosition = reader.getPosition();
|
||||
}
|
||||
if (lastPosition > lastSplitStart) {
|
||||
FileSplit split = new FileSplit(listingFilePath, lastSplitStart,
|
||||
lastPosition - lastSplitStart, null);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.info ("Creating split : " + split + ", bytes in split: " + currentSplitSize);
|
||||
}
|
||||
splits.add(split);
|
||||
}
|
||||
|
||||
} finally {
|
||||
IOUtils.closeStream(reader);
|
||||
}
|
||||
|
||||
return splits;
|
||||
}
|
||||
|
||||
private static Path getListingFilePath(Configuration configuration) {
|
||||
final String listingFilePathString =
|
||||
configuration.get(DistCpConstants.CONF_LABEL_LISTING_FILE_PATH, "");
|
||||
|
||||
assert !listingFilePathString.equals("")
|
||||
: "Couldn't find listing file. Invalid input.";
|
||||
return new Path(listingFilePathString);
|
||||
}
|
||||
|
||||
private SequenceFile.Reader getListingFileReader(Configuration configuration) {
|
||||
|
||||
final Path listingFilePath = getListingFilePath(configuration);
|
||||
try {
|
||||
final FileSystem fileSystem = listingFilePath.getFileSystem(configuration);
|
||||
if (!fileSystem.exists(listingFilePath))
|
||||
throw new IllegalArgumentException("Listing file doesn't exist at: "
|
||||
+ listingFilePath);
|
||||
|
||||
return new SequenceFile.Reader(configuration,
|
||||
SequenceFile.Reader.file(listingFilePath));
|
||||
}
|
||||
catch (IOException exception) {
|
||||
LOG.error("Couldn't find listing file at: " + listingFilePath, exception);
|
||||
throw new IllegalArgumentException("Couldn't find listing-file at: "
|
||||
+ listingFilePath, exception);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of InputFormat::createRecordReader().
|
||||
* @param split The split for which the RecordReader is sought.
|
||||
* @param context The context of the current task-attempt.
|
||||
* @return A SequenceFileRecordReader instance, (since the copy-listing is a
|
||||
* simple sequence-file.)
|
||||
* @throws IOException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public RecordReader<Text, FileStatus> createRecordReader(InputSplit split,
|
||||
TaskAttemptContext context)
|
||||
throws IOException, InterruptedException {
|
||||
return new SequenceFileRecordReader<Text, FileStatus>();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,246 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred.lib;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader;
|
||||
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hadoop.mapreduce.TaskID;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* The DynamicInputChunk represents a single chunk of work, when used in
|
||||
* conjunction with the DynamicInputFormat and the DynamicRecordReader.
|
||||
* The records in the DynamicInputFormat's input-file are split across various
|
||||
* DynamicInputChunks. Each one is claimed and processed in an iteration of
|
||||
* a dynamic-mapper. When a DynamicInputChunk has been exhausted, the faster
|
||||
* mapper may claim another and process it, until there are no more to be
|
||||
* consumed.
|
||||
*/
|
||||
class DynamicInputChunk<K, V> {
|
||||
private static Log LOG = LogFactory.getLog(DynamicInputChunk.class);
|
||||
|
||||
private static Configuration configuration;
|
||||
private static Path chunkRootPath;
|
||||
private static String chunkFilePrefix;
|
||||
private static int numChunksLeft = -1; // Un-initialized before 1st dir-scan.
|
||||
private static FileSystem fs;
|
||||
|
||||
private Path chunkFilePath;
|
||||
private SequenceFileRecordReader<K, V> reader;
|
||||
private SequenceFile.Writer writer;
|
||||
|
||||
private static void initializeChunkInvariants(Configuration config)
|
||||
throws IOException {
|
||||
configuration = config;
|
||||
Path listingFilePath = new Path(getListingFilePath(configuration));
|
||||
chunkRootPath = new Path(listingFilePath.getParent(), "chunkDir");
|
||||
fs = chunkRootPath.getFileSystem(configuration);
|
||||
chunkFilePrefix = listingFilePath.getName() + ".chunk.";
|
||||
}
|
||||
|
||||
private static String getListingFilePath(Configuration configuration) {
|
||||
final String listingFileString = configuration.get(
|
||||
DistCpConstants.CONF_LABEL_LISTING_FILE_PATH, "");
|
||||
assert !listingFileString.equals("") : "Listing file not found.";
|
||||
return listingFileString;
|
||||
}
|
||||
|
||||
private static boolean areInvariantsInitialized() {
|
||||
return chunkRootPath != null;
|
||||
}
|
||||
|
||||
private DynamicInputChunk(String chunkId, Configuration configuration)
|
||||
throws IOException {
|
||||
if (!areInvariantsInitialized())
|
||||
initializeChunkInvariants(configuration);
|
||||
|
||||
chunkFilePath = new Path(chunkRootPath, chunkFilePrefix + chunkId);
|
||||
openForWrite();
|
||||
}
|
||||
|
||||
|
||||
private void openForWrite() throws IOException {
|
||||
writer = SequenceFile.createWriter(
|
||||
chunkFilePath.getFileSystem(configuration), configuration,
|
||||
chunkFilePath, Text.class, FileStatus.class,
|
||||
SequenceFile.CompressionType.NONE);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Factory method to create chunk-files for writing to.
|
||||
* (For instance, when the DynamicInputFormat splits the input-file into
|
||||
* chunks.)
|
||||
* @param chunkId String to identify the chunk.
|
||||
* @param configuration Configuration, describing the location of the listing-
|
||||
* file, file-system for the map-job, etc.
|
||||
* @return A DynamicInputChunk, corresponding to a chunk-file, with the name
|
||||
* incorporating the chunk-id.
|
||||
* @throws IOException Exception on failure to create the chunk.
|
||||
*/
|
||||
public static DynamicInputChunk createChunkForWrite(String chunkId,
|
||||
Configuration configuration) throws IOException {
|
||||
return new DynamicInputChunk(chunkId, configuration);
|
||||
}
|
||||
|
||||
/**
|
||||
* Method to write records into a chunk.
|
||||
* @param key Key from the listing file.
|
||||
* @param value Corresponding value from the listing file.
|
||||
* @throws IOException Exception onf failure to write to the file.
|
||||
*/
|
||||
public void write(Text key, FileStatus value) throws IOException {
|
||||
writer.append(key, value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes streams opened to the chunk-file.
|
||||
*/
|
||||
public void close() {
|
||||
IOUtils.cleanup(LOG, reader, writer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reassigns the chunk to a specified Map-Task, for consumption.
|
||||
* @param taskId The Map-Task to which a the chunk is to be reassigned.
|
||||
* @throws IOException Exception on failure to reassign.
|
||||
*/
|
||||
public void assignTo(TaskID taskId) throws IOException {
|
||||
Path newPath = new Path(chunkRootPath, taskId.toString());
|
||||
if (!fs.rename(chunkFilePath, newPath)) {
|
||||
LOG.warn(chunkFilePath + " could not be assigned to " + taskId);
|
||||
}
|
||||
}
|
||||
|
||||
private DynamicInputChunk(Path chunkFilePath,
|
||||
TaskAttemptContext taskAttemptContext)
|
||||
throws IOException, InterruptedException {
|
||||
if (!areInvariantsInitialized())
|
||||
initializeChunkInvariants(taskAttemptContext.getConfiguration());
|
||||
|
||||
this.chunkFilePath = chunkFilePath;
|
||||
openForRead(taskAttemptContext);
|
||||
}
|
||||
|
||||
private void openForRead(TaskAttemptContext taskAttemptContext)
|
||||
throws IOException, InterruptedException {
|
||||
reader = new SequenceFileRecordReader<K, V>();
|
||||
reader.initialize(new FileSplit(chunkFilePath, 0,
|
||||
DistCpUtils.getFileSize(chunkFilePath, configuration), null),
|
||||
taskAttemptContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Factory method that
|
||||
* 1. acquires a chunk for the specified map-task attempt
|
||||
* 2. returns a DynamicInputChunk associated with the acquired chunk-file.
|
||||
* @param taskAttemptContext The attempt-context for the map task that's
|
||||
* trying to acquire a chunk.
|
||||
* @return The acquired dynamic-chunk. The chunk-file is renamed to the
|
||||
* attempt-id (from the attempt-context.)
|
||||
* @throws IOException Exception on failure.
|
||||
* @throws InterruptedException Exception on failure.
|
||||
*/
|
||||
public static DynamicInputChunk acquire(TaskAttemptContext taskAttemptContext)
|
||||
throws IOException, InterruptedException {
|
||||
if (!areInvariantsInitialized())
|
||||
initializeChunkInvariants(taskAttemptContext.getConfiguration());
|
||||
|
||||
String taskId
|
||||
= taskAttemptContext.getTaskAttemptID().getTaskID().toString();
|
||||
Path acquiredFilePath = new Path(chunkRootPath, taskId);
|
||||
|
||||
if (fs.exists(acquiredFilePath)) {
|
||||
LOG.info("Acquiring pre-assigned chunk: " + acquiredFilePath);
|
||||
return new DynamicInputChunk(acquiredFilePath, taskAttemptContext);
|
||||
}
|
||||
|
||||
for (FileStatus chunkFile : getListOfChunkFiles()) {
|
||||
if (fs.rename(chunkFile.getPath(), acquiredFilePath)) {
|
||||
LOG.info(taskId + " acquired " + chunkFile.getPath());
|
||||
return new DynamicInputChunk(acquiredFilePath, taskAttemptContext);
|
||||
}
|
||||
else
|
||||
LOG.warn(taskId + " could not acquire " + chunkFile.getPath());
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Method to be called to relinquish an acquired chunk. All streams open to
|
||||
* the chunk are closed, and the chunk-file is deleted.
|
||||
* @throws IOException Exception thrown on failure to release (i.e. delete)
|
||||
* the chunk file.
|
||||
*/
|
||||
public void release() throws IOException {
|
||||
close();
|
||||
if (!fs.delete(chunkFilePath, false)) {
|
||||
LOG.error("Unable to release chunk at path: " + chunkFilePath);
|
||||
throw new IOException("Unable to release chunk at path: " + chunkFilePath);
|
||||
}
|
||||
}
|
||||
|
||||
static FileStatus [] getListOfChunkFiles() throws IOException {
|
||||
Path chunkFilePattern = new Path(chunkRootPath, chunkFilePrefix + "*");
|
||||
FileStatus chunkFiles[] = fs.globStatus(chunkFilePattern);
|
||||
numChunksLeft = chunkFiles.length;
|
||||
return chunkFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for the chunk-file's path, on HDFS.
|
||||
* @return The qualified path to the chunk-file.
|
||||
*/
|
||||
public Path getPath() {
|
||||
return chunkFilePath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for the record-reader, opened to the chunk-file.
|
||||
* @return Opened Sequence-file reader.
|
||||
*/
|
||||
public SequenceFileRecordReader<K,V> getReader() {
|
||||
assert reader != null : "Reader un-initialized!";
|
||||
return reader;
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for the number of chunk-files left in the chunk-file directory.
|
||||
* Useful to determine how many chunks (and hence, records) are left to be
|
||||
* processed.
|
||||
* @return Before the first scan of the directory, the number returned is -1.
|
||||
* Otherwise, the number of chunk-files seen from the last scan is returned.
|
||||
*/
|
||||
public static int getNumChunksLeft() {
|
||||
return numChunksLeft;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,292 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred.lib;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* DynamicInputFormat implements the "Worker pattern" for DistCp.
|
||||
* Rather than to split up the copy-list into a set of static splits,
|
||||
* the DynamicInputFormat does the following:
|
||||
* 1. Splits the copy-list into small chunks on the DFS.
|
||||
* 2. Creates a set of empty "dynamic" splits, that each consume as many chunks
|
||||
* as it can.
|
||||
* This arrangement ensures that a single slow mapper won't slow down the entire
|
||||
* job (since the slack will be picked up by other mappers, who consume more
|
||||
* chunks.)
|
||||
* By varying the split-ratio, one can vary chunk sizes to achieve different
|
||||
* performance characteristics.
|
||||
*/
|
||||
public class DynamicInputFormat<K, V> extends InputFormat<K, V> {
|
||||
private static final Log LOG = LogFactory.getLog(DynamicInputFormat.class);
|
||||
|
||||
private static final String CONF_LABEL_LISTING_SPLIT_RATIO
|
||||
= "mapred.listing.split.ratio";
|
||||
private static final String CONF_LABEL_NUM_SPLITS
|
||||
= "mapred.num.splits";
|
||||
private static final String CONF_LABEL_NUM_ENTRIES_PER_CHUNK
|
||||
= "mapred.num.entries.per.chunk";
|
||||
|
||||
/**
|
||||
* Implementation of InputFormat::getSplits(). This method splits up the
|
||||
* copy-listing file into chunks, and assigns the first batch to different
|
||||
* tasks.
|
||||
* @param jobContext JobContext for the map job.
|
||||
* @return The list of (empty) dynamic input-splits.
|
||||
* @throws IOException, on failure.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public List<InputSplit> getSplits(JobContext jobContext)
|
||||
throws IOException, InterruptedException {
|
||||
LOG.info("DynamicInputFormat: Getting splits for job:"
|
||||
+ jobContext.getJobID());
|
||||
return createSplits(jobContext,
|
||||
splitCopyListingIntoChunksWithShuffle(jobContext));
|
||||
}
|
||||
|
||||
private List<InputSplit> createSplits(JobContext jobContext,
|
||||
List<DynamicInputChunk> chunks)
|
||||
throws IOException {
|
||||
int numMaps = getNumMapTasks(jobContext.getConfiguration());
|
||||
|
||||
final int nSplits = Math.min(numMaps, chunks.size());
|
||||
List<InputSplit> splits = new ArrayList<InputSplit>(nSplits);
|
||||
|
||||
for (int i=0; i< nSplits; ++i) {
|
||||
TaskID taskId = new TaskID(jobContext.getJobID(), TaskType.MAP, i);
|
||||
chunks.get(i).assignTo(taskId);
|
||||
splits.add(new FileSplit(chunks.get(i).getPath(), 0,
|
||||
// Setting non-zero length for FileSplit size, to avoid a possible
|
||||
// future when 0-sized file-splits are considered "empty" and skipped
|
||||
// over.
|
||||
MIN_RECORDS_PER_CHUNK,
|
||||
null));
|
||||
}
|
||||
DistCpUtils.publish(jobContext.getConfiguration(),
|
||||
CONF_LABEL_NUM_SPLITS, splits.size());
|
||||
return splits;
|
||||
}
|
||||
|
||||
private static int N_CHUNKS_OPEN_AT_ONCE_DEFAULT = 16;
|
||||
|
||||
private List<DynamicInputChunk> splitCopyListingIntoChunksWithShuffle
|
||||
(JobContext context) throws IOException {
|
||||
|
||||
final Configuration configuration = context.getConfiguration();
|
||||
int numRecords = getNumberOfRecords(configuration);
|
||||
int numMaps = getNumMapTasks(configuration);
|
||||
// Number of chunks each map will process, on average.
|
||||
int splitRatio = getListingSplitRatio(configuration, numMaps, numRecords);
|
||||
validateNumChunksUsing(splitRatio, numMaps);
|
||||
|
||||
int numEntriesPerChunk = (int)Math.ceil((float)numRecords
|
||||
/(splitRatio * numMaps));
|
||||
DistCpUtils.publish(context.getConfiguration(),
|
||||
CONF_LABEL_NUM_ENTRIES_PER_CHUNK,
|
||||
numEntriesPerChunk);
|
||||
|
||||
final int nChunksTotal = (int)Math.ceil((float)numRecords/numEntriesPerChunk);
|
||||
int nChunksOpenAtOnce
|
||||
= Math.min(N_CHUNKS_OPEN_AT_ONCE_DEFAULT, nChunksTotal);
|
||||
|
||||
Path listingPath = getListingFilePath(configuration);
|
||||
SequenceFile.Reader reader
|
||||
= new SequenceFile.Reader(configuration,
|
||||
SequenceFile.Reader.file(listingPath));
|
||||
|
||||
List<DynamicInputChunk> openChunks
|
||||
= new ArrayList<DynamicInputChunk>();
|
||||
|
||||
List<DynamicInputChunk> chunksFinal = new ArrayList<DynamicInputChunk>();
|
||||
|
||||
FileStatus fileStatus = new FileStatus();
|
||||
Text relPath = new Text();
|
||||
int recordCounter = 0;
|
||||
int chunkCount = 0;
|
||||
|
||||
try {
|
||||
|
||||
while (reader.next(relPath, fileStatus)) {
|
||||
if (recordCounter % (nChunksOpenAtOnce*numEntriesPerChunk) == 0) {
|
||||
// All chunks full. Create new chunk-set.
|
||||
closeAll(openChunks);
|
||||
chunksFinal.addAll(openChunks);
|
||||
|
||||
openChunks = createChunks(
|
||||
configuration, chunkCount, nChunksTotal, nChunksOpenAtOnce);
|
||||
|
||||
chunkCount += openChunks.size();
|
||||
|
||||
nChunksOpenAtOnce = openChunks.size();
|
||||
recordCounter = 0;
|
||||
}
|
||||
|
||||
// Shuffle into open chunks.
|
||||
openChunks.get(recordCounter%nChunksOpenAtOnce).write(relPath, fileStatus);
|
||||
++recordCounter;
|
||||
}
|
||||
|
||||
} finally {
|
||||
closeAll(openChunks);
|
||||
chunksFinal.addAll(openChunks);
|
||||
IOUtils.closeStream(reader);
|
||||
}
|
||||
|
||||
LOG.info("Number of dynamic-chunk-files created: " + chunksFinal.size());
|
||||
return chunksFinal;
|
||||
}
|
||||
|
||||
private static void validateNumChunksUsing(int splitRatio, int numMaps)
|
||||
throws IOException {
|
||||
if (splitRatio * numMaps > MAX_CHUNKS_TOLERABLE)
|
||||
throw new IOException("Too many chunks created with splitRatio:"
|
||||
+ splitRatio + ", numMaps:" + numMaps
|
||||
+ ". Reduce numMaps or decrease split-ratio to proceed.");
|
||||
}
|
||||
|
||||
private static void closeAll(List<DynamicInputChunk> chunks) {
|
||||
for (DynamicInputChunk chunk: chunks)
|
||||
chunk.close();
|
||||
}
|
||||
|
||||
private static List<DynamicInputChunk> createChunks(Configuration config,
|
||||
int chunkCount, int nChunksTotal, int nChunksOpenAtOnce)
|
||||
throws IOException {
|
||||
List<DynamicInputChunk> chunks = new ArrayList<DynamicInputChunk>();
|
||||
int chunkIdUpperBound
|
||||
= Math.min(nChunksTotal, chunkCount + nChunksOpenAtOnce);
|
||||
|
||||
// If there will be fewer than nChunksOpenAtOnce chunks left after
|
||||
// the current batch of chunks, fold the remaining chunks into
|
||||
// the current batch.
|
||||
if (nChunksTotal - chunkIdUpperBound < nChunksOpenAtOnce)
|
||||
chunkIdUpperBound = nChunksTotal;
|
||||
|
||||
for (int i=chunkCount; i < chunkIdUpperBound; ++i)
|
||||
chunks.add(createChunk(i, config));
|
||||
return chunks;
|
||||
}
|
||||
|
||||
private static DynamicInputChunk createChunk(int chunkId, Configuration config)
|
||||
throws IOException {
|
||||
return DynamicInputChunk.createChunkForWrite(String.format("%05d", chunkId),
|
||||
config);
|
||||
}
|
||||
|
||||
|
||||
private static Path getListingFilePath(Configuration configuration) {
|
||||
String listingFilePathString = configuration.get(
|
||||
DistCpConstants.CONF_LABEL_LISTING_FILE_PATH, "");
|
||||
|
||||
assert !listingFilePathString.equals("") : "Listing file not found.";
|
||||
|
||||
Path listingFilePath = new Path(listingFilePathString);
|
||||
try {
|
||||
assert listingFilePath.getFileSystem(configuration)
|
||||
.exists(listingFilePath) : "Listing file: " + listingFilePath +
|
||||
" not found.";
|
||||
} catch (IOException e) {
|
||||
assert false : "Listing file: " + listingFilePath
|
||||
+ " couldn't be accessed. " + e.getMessage();
|
||||
}
|
||||
return listingFilePath;
|
||||
}
|
||||
|
||||
private static int getNumberOfRecords(Configuration configuration) {
|
||||
return DistCpUtils.getInt(configuration,
|
||||
DistCpConstants.CONF_LABEL_TOTAL_NUMBER_OF_RECORDS);
|
||||
}
|
||||
|
||||
private static int getNumMapTasks(Configuration configuration) {
|
||||
return DistCpUtils.getInt(configuration,
|
||||
JobContext.NUM_MAPS);
|
||||
}
|
||||
|
||||
private static int getListingSplitRatio(Configuration configuration,
|
||||
int numMaps, int numPaths) {
|
||||
return configuration.getInt(
|
||||
CONF_LABEL_LISTING_SPLIT_RATIO,
|
||||
getSplitRatio(numMaps, numPaths));
|
||||
}
|
||||
|
||||
private static final int MAX_CHUNKS_TOLERABLE = 400;
|
||||
private static final int MAX_CHUNKS_IDEAL = 100;
|
||||
private static final int MIN_RECORDS_PER_CHUNK = 5;
|
||||
private static final int SPLIT_RATIO_DEFAULT = 2;
|
||||
|
||||
/**
|
||||
* Package private, for testability.
|
||||
* @param nMaps The number of maps requested for.
|
||||
* @param nRecords The number of records to be copied.
|
||||
* @return The number of splits each map should handle, ideally.
|
||||
*/
|
||||
static int getSplitRatio(int nMaps, int nRecords) {
|
||||
if (nMaps == 1) {
|
||||
LOG.warn("nMaps == 1. Why use DynamicInputFormat?");
|
||||
return 1;
|
||||
}
|
||||
|
||||
if (nMaps > MAX_CHUNKS_IDEAL)
|
||||
return SPLIT_RATIO_DEFAULT;
|
||||
|
||||
int nPickups = (int)Math.ceil((float)MAX_CHUNKS_IDEAL/nMaps);
|
||||
int nRecordsPerChunk = (int)Math.ceil((float)nRecords/(nMaps*nPickups));
|
||||
|
||||
return nRecordsPerChunk < MIN_RECORDS_PER_CHUNK ?
|
||||
SPLIT_RATIO_DEFAULT : nPickups;
|
||||
}
|
||||
|
||||
static int getNumEntriesPerChunk(Configuration configuration) {
|
||||
return DistCpUtils.getInt(configuration,
|
||||
CONF_LABEL_NUM_ENTRIES_PER_CHUNK);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Implementation of Inputformat::createRecordReader().
|
||||
* @param inputSplit The split for which the RecordReader is required.
|
||||
* @param taskAttemptContext TaskAttemptContext for the current attempt.
|
||||
* @return DynamicRecordReader instance.
|
||||
* @throws IOException, on failure.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public RecordReader<K, V> createRecordReader(
|
||||
InputSplit inputSplit,
|
||||
TaskAttemptContext taskAttemptContext)
|
||||
throws IOException, InterruptedException {
|
||||
return new DynamicRecordReader<K, V>();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,203 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred.lib;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* The DynamicRecordReader is used in conjunction with the DynamicInputFormat
|
||||
* to implement the "Worker pattern" for DistCp.
|
||||
* The DynamicRecordReader is responsible for:
|
||||
* 1. Presenting the contents of each chunk to DistCp's mapper.
|
||||
* 2. Acquiring a new chunk when the current chunk has been completely consumed,
|
||||
* transparently.
|
||||
*/
|
||||
public class DynamicRecordReader<K, V> extends RecordReader<K, V> {
|
||||
private static final Log LOG = LogFactory.getLog(DynamicRecordReader.class);
|
||||
private TaskAttemptContext taskAttemptContext;
|
||||
private Configuration configuration;
|
||||
private DynamicInputChunk<K, V> chunk;
|
||||
private TaskID taskId;
|
||||
|
||||
// Data required for progress indication.
|
||||
private int numRecordsPerChunk; // Constant per job.
|
||||
private int totalNumRecords; // Constant per job.
|
||||
private int numRecordsProcessedByThisMap = 0;
|
||||
private long timeOfLastChunkDirScan = 0;
|
||||
private boolean isChunkDirAlreadyScanned = false;
|
||||
|
||||
private static long TIME_THRESHOLD_FOR_DIR_SCANS = TimeUnit.MINUTES.toMillis(5);
|
||||
|
||||
/**
|
||||
* Implementation for RecordReader::initialize(). Initializes the internal
|
||||
* RecordReader to read from chunks.
|
||||
* @param inputSplit The InputSplit for the map. Ignored entirely.
|
||||
* @param taskAttemptContext The AttemptContext.
|
||||
* @throws IOException, on failure.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public void initialize(InputSplit inputSplit,
|
||||
TaskAttemptContext taskAttemptContext)
|
||||
throws IOException, InterruptedException {
|
||||
numRecordsPerChunk = DynamicInputFormat.getNumEntriesPerChunk(
|
||||
taskAttemptContext.getConfiguration());
|
||||
this.taskAttemptContext = taskAttemptContext;
|
||||
configuration = taskAttemptContext.getConfiguration();
|
||||
taskId = taskAttemptContext.getTaskAttemptID().getTaskID();
|
||||
chunk = DynamicInputChunk.acquire(this.taskAttemptContext);
|
||||
timeOfLastChunkDirScan = System.currentTimeMillis();
|
||||
isChunkDirAlreadyScanned = false;
|
||||
|
||||
totalNumRecords = getTotalNumRecords();
|
||||
|
||||
}
|
||||
|
||||
private int getTotalNumRecords() {
|
||||
return DistCpUtils.getInt(configuration,
|
||||
DistCpConstants.CONF_LABEL_TOTAL_NUMBER_OF_RECORDS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of RecordReader::nextValue().
|
||||
* Reads the contents of the current chunk and returns them. When a chunk has
|
||||
* been completely exhausted, an new chunk is acquired and read,
|
||||
* transparently.
|
||||
* @return True, if the nextValue() could be traversed to. False, otherwise.
|
||||
* @throws IOException, on failure.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public boolean nextKeyValue()
|
||||
throws IOException, InterruptedException {
|
||||
|
||||
if (chunk == null) {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug(taskId + ": RecordReader is null. No records to be read.");
|
||||
return false;
|
||||
}
|
||||
|
||||
if (chunk.getReader().nextKeyValue()) {
|
||||
++numRecordsProcessedByThisMap;
|
||||
return true;
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug(taskId + ": Current chunk exhausted. " +
|
||||
" Attempting to pick up new one.");
|
||||
|
||||
chunk.release();
|
||||
timeOfLastChunkDirScan = System.currentTimeMillis();
|
||||
isChunkDirAlreadyScanned = false;
|
||||
|
||||
chunk = DynamicInputChunk.acquire(taskAttemptContext);
|
||||
|
||||
if (chunk == null) return false;
|
||||
|
||||
if (chunk.getReader().nextKeyValue()) {
|
||||
++numRecordsProcessedByThisMap;
|
||||
return true;
|
||||
}
|
||||
else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of RecordReader::getCurrentKey().
|
||||
* @return The key of the current record. (i.e. the source-path.)
|
||||
* @throws IOException, on failure.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public K getCurrentKey()
|
||||
throws IOException, InterruptedException {
|
||||
return chunk.getReader().getCurrentKey();
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of RecordReader::getCurrentValue().
|
||||
* @return The value of the current record. (i.e. the target-path.)
|
||||
* @throws IOException, on failure.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public V getCurrentValue()
|
||||
throws IOException, InterruptedException {
|
||||
return chunk.getReader().getCurrentValue();
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of RecordReader::getProgress().
|
||||
* @return A fraction [0.0,1.0] indicating the progress of a DistCp mapper.
|
||||
* @throws IOException, on failure.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@Override
|
||||
public float getProgress()
|
||||
throws IOException, InterruptedException {
|
||||
final int numChunksLeft = getNumChunksLeft();
|
||||
if (numChunksLeft < 0) {// Un-initialized. i.e. Before 1st dir-scan.
|
||||
assert numRecordsProcessedByThisMap <= numRecordsPerChunk
|
||||
: "numRecordsProcessedByThisMap:" + numRecordsProcessedByThisMap +
|
||||
" exceeds numRecordsPerChunk:" + numRecordsPerChunk;
|
||||
return ((float) numRecordsProcessedByThisMap) / totalNumRecords;
|
||||
// Conservative estimate, till the first directory scan.
|
||||
}
|
||||
|
||||
return ((float) numRecordsProcessedByThisMap)
|
||||
/(numRecordsProcessedByThisMap + numRecordsPerChunk*numChunksLeft);
|
||||
}
|
||||
|
||||
private int getNumChunksLeft() throws IOException {
|
||||
long now = System.currentTimeMillis();
|
||||
boolean tooLongSinceLastDirScan
|
||||
= now - timeOfLastChunkDirScan > TIME_THRESHOLD_FOR_DIR_SCANS;
|
||||
|
||||
if (tooLongSinceLastDirScan
|
||||
|| (!isChunkDirAlreadyScanned &&
|
||||
numRecordsProcessedByThisMap%numRecordsPerChunk
|
||||
> numRecordsPerChunk/2)) {
|
||||
DynamicInputChunk.getListOfChunkFiles();
|
||||
isChunkDirAlreadyScanned = true;
|
||||
timeOfLastChunkDirScan = now;
|
||||
}
|
||||
|
||||
return DynamicInputChunk.getNumChunksLeft();
|
||||
}
|
||||
/**
|
||||
* Implementation of RecordReader::close().
|
||||
* Closes the RecordReader.
|
||||
* @throws IOException, on failure.
|
||||
*/
|
||||
@Override
|
||||
public void close()
|
||||
throws IOException {
|
||||
if (chunk != null)
|
||||
chunk.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,343 @@
|
|||
/**
|
||||
* 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.hadoop.tools.util;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileChecksum;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
||||
import org.apache.hadoop.tools.mapred.UniformSizeInputFormat;
|
||||
import org.apache.hadoop.tools.DistCpOptions;
|
||||
import org.apache.hadoop.mapreduce.InputFormat;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.Locale;
|
||||
import java.text.DecimalFormat;
|
||||
import java.net.URI;
|
||||
import java.net.InetAddress;
|
||||
import java.net.UnknownHostException;
|
||||
|
||||
/**
|
||||
* Utility functions used in DistCp.
|
||||
*/
|
||||
public class DistCpUtils {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(DistCpUtils.class);
|
||||
|
||||
/**
|
||||
* Retrieves size of the file at the specified path.
|
||||
* @param path The path of the file whose size is sought.
|
||||
* @param configuration Configuration, to retrieve the appropriate FileSystem.
|
||||
* @return The file-size, in number of bytes.
|
||||
* @throws IOException, on failure.
|
||||
*/
|
||||
public static long getFileSize(Path path, Configuration configuration)
|
||||
throws IOException {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug("Retrieving file size for: " + path);
|
||||
return path.getFileSystem(configuration).getFileStatus(path).getLen();
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to publish a value to a configuration.
|
||||
* @param configuration The Configuration to which the value must be written.
|
||||
* @param label The label for the value being published.
|
||||
* @param value The value being published.
|
||||
* @param <T> The type of the value.
|
||||
*/
|
||||
public static <T> void publish(Configuration configuration,
|
||||
String label, T value) {
|
||||
configuration.set(label, String.valueOf(value));
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to retrieve a specified key from a Configuration. Throw exception
|
||||
* if not found.
|
||||
* @param configuration The Configuration in which the key is sought.
|
||||
* @param label The key being sought.
|
||||
* @return Integer value of the key.
|
||||
*/
|
||||
public static int getInt(Configuration configuration, String label) {
|
||||
int value = configuration.getInt(label, -1);
|
||||
assert value >= 0 : "Couldn't find " + label;
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to retrieve a specified key from a Configuration. Throw exception
|
||||
* if not found.
|
||||
* @param configuration The Configuration in which the key is sought.
|
||||
* @param label The key being sought.
|
||||
* @return Long value of the key.
|
||||
*/
|
||||
public static long getLong(Configuration configuration, String label) {
|
||||
long value = configuration.getLong(label, -1);
|
||||
assert value >= 0 : "Couldn't find " + label;
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the class that implements a copy strategy. Looks up the implementation for
|
||||
* a particular strategy from distcp-default.xml
|
||||
*
|
||||
* @param conf - Configuration object
|
||||
* @param options - Handle to input options
|
||||
* @return Class implementing the strategy specified in options.
|
||||
*/
|
||||
public static Class<? extends InputFormat> getStrategy(Configuration conf,
|
||||
DistCpOptions options) {
|
||||
String confLabel = "distcp." +
|
||||
options.getCopyStrategy().toLowerCase(Locale.getDefault()) + ".strategy.impl";
|
||||
return conf.getClass(confLabel, UniformSizeInputFormat.class, InputFormat.class);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets relative path of child path with respect to a root path
|
||||
* For ex. If childPath = /tmp/abc/xyz/file and
|
||||
* sourceRootPath = /tmp/abc
|
||||
* Relative path would be /xyz/file
|
||||
* If childPath = /file and
|
||||
* sourceRootPath = /
|
||||
* Relative path would be /file
|
||||
* @param sourceRootPath - Source root path
|
||||
* @param childPath - Path for which relative path is required
|
||||
* @return - Relative portion of the child path (always prefixed with /
|
||||
* unless it is empty
|
||||
*/
|
||||
public static String getRelativePath(Path sourceRootPath, Path childPath) {
|
||||
String childPathString = childPath.toUri().getPath();
|
||||
String sourceRootPathString = sourceRootPath.toUri().getPath();
|
||||
return sourceRootPathString.equals("/") ? childPathString :
|
||||
childPathString.substring(sourceRootPathString.length());
|
||||
}
|
||||
|
||||
/**
|
||||
* Pack file preservation attributes into a string, containing
|
||||
* just the first character of each preservation attribute
|
||||
* @param attributes - Attribute set to preserve
|
||||
* @return - String containing first letters of each attribute to preserve
|
||||
*/
|
||||
public static String packAttributes(EnumSet<FileAttribute> attributes) {
|
||||
StringBuffer buffer = new StringBuffer(5);
|
||||
int len = 0;
|
||||
for (FileAttribute attribute : attributes) {
|
||||
buffer.append(attribute.name().charAt(0));
|
||||
len++;
|
||||
}
|
||||
return buffer.substring(0, len);
|
||||
}
|
||||
|
||||
/**
|
||||
* Un packs preservation attribute string containing the first character of
|
||||
* each preservation attribute back to a set of attributes to preserve
|
||||
* @param attributes - Attribute string
|
||||
* @return - Attribute set
|
||||
*/
|
||||
public static EnumSet<FileAttribute> unpackAttributes(String attributes) {
|
||||
EnumSet<FileAttribute> retValue = EnumSet.noneOf(FileAttribute.class);
|
||||
|
||||
if (attributes != null) {
|
||||
for (int index = 0; index < attributes.length(); index++) {
|
||||
retValue.add(FileAttribute.getAttribute(attributes.charAt(index)));
|
||||
}
|
||||
}
|
||||
|
||||
return retValue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Preserve attribute on file matching that of the file status being sent
|
||||
* as argument. Barring the block size, all the other attributes are preserved
|
||||
* by this function
|
||||
*
|
||||
* @param targetFS - File system
|
||||
* @param path - Path that needs to preserve original file status
|
||||
* @param srcFileStatus - Original file status
|
||||
* @param attributes - Attribute set that need to be preserved
|
||||
* @throws IOException - Exception if any (particularly relating to group/owner
|
||||
* change or any transient error)
|
||||
*/
|
||||
public static void preserve(FileSystem targetFS, Path path,
|
||||
FileStatus srcFileStatus,
|
||||
EnumSet<FileAttribute> attributes) throws IOException {
|
||||
|
||||
FileStatus targetFileStatus = targetFS.getFileStatus(path);
|
||||
String group = targetFileStatus.getGroup();
|
||||
String user = targetFileStatus.getOwner();
|
||||
boolean chown = false;
|
||||
|
||||
if (attributes.contains(FileAttribute.PERMISSION) &&
|
||||
!srcFileStatus.getPermission().equals(targetFileStatus.getPermission())) {
|
||||
targetFS.setPermission(path, srcFileStatus.getPermission());
|
||||
}
|
||||
|
||||
if (attributes.contains(FileAttribute.REPLICATION) && ! targetFileStatus.isDirectory() &&
|
||||
srcFileStatus.getReplication() != targetFileStatus.getReplication()) {
|
||||
targetFS.setReplication(path, srcFileStatus.getReplication());
|
||||
}
|
||||
|
||||
if (attributes.contains(FileAttribute.GROUP) &&
|
||||
!group.equals(srcFileStatus.getGroup())) {
|
||||
group = srcFileStatus.getGroup();
|
||||
chown = true;
|
||||
}
|
||||
|
||||
if (attributes.contains(FileAttribute.USER) &&
|
||||
!user.equals(srcFileStatus.getOwner())) {
|
||||
user = srcFileStatus.getOwner();
|
||||
chown = true;
|
||||
}
|
||||
|
||||
if (chown) {
|
||||
targetFS.setOwner(path, user, group);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort sequence file containing FileStatus and Text as key and value respecitvely
|
||||
*
|
||||
* @param fs - File System
|
||||
* @param conf - Configuration
|
||||
* @param sourceListing - Source listing file
|
||||
* @return Path of the sorted file. Is source file with _sorted appended to the name
|
||||
* @throws IOException - Any exception during sort.
|
||||
*/
|
||||
public static Path sortListing(FileSystem fs, Configuration conf, Path sourceListing)
|
||||
throws IOException {
|
||||
SequenceFile.Sorter sorter = new SequenceFile.Sorter(fs, Text.class, FileStatus.class, conf);
|
||||
Path output = new Path(sourceListing.toString() + "_sorted");
|
||||
|
||||
if (fs.exists(output)) {
|
||||
fs.delete(output, false);
|
||||
}
|
||||
|
||||
sorter.sort(sourceListing, output);
|
||||
return output;
|
||||
}
|
||||
|
||||
/**
|
||||
* String utility to convert a number-of-bytes to human readable format.
|
||||
*/
|
||||
private static ThreadLocal<DecimalFormat> FORMATTER
|
||||
= new ThreadLocal<DecimalFormat>() {
|
||||
@Override
|
||||
protected DecimalFormat initialValue() {
|
||||
return new DecimalFormat("0.0");
|
||||
}
|
||||
};
|
||||
|
||||
public static DecimalFormat getFormatter() {
|
||||
return FORMATTER.get();
|
||||
}
|
||||
|
||||
public static String getStringDescriptionFor(long nBytes) {
|
||||
|
||||
char units [] = {'B', 'K', 'M', 'G', 'T', 'P'};
|
||||
|
||||
double current = nBytes;
|
||||
double prev = current;
|
||||
int index = 0;
|
||||
|
||||
while ((current = current/1024) >= 1) {
|
||||
prev = current;
|
||||
++index;
|
||||
}
|
||||
|
||||
assert index < units.length : "Too large a number.";
|
||||
|
||||
return getFormatter().format(prev) + units[index];
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to compare checksums for the paths specified.
|
||||
*
|
||||
* If checksums's can't be retrieved, it doesn't fail the test
|
||||
* Only time the comparison would fail is when checksums are
|
||||
* available and they don't match
|
||||
*
|
||||
* @param sourceFS FileSystem for the source path.
|
||||
* @param source The source path.
|
||||
* @param targetFS FileSystem for the target path.
|
||||
* @param target The target path.
|
||||
* @return If either checksum couldn't be retrieved, the function returns
|
||||
* false. If checksums are retrieved, the function returns true if they match,
|
||||
* and false otherwise.
|
||||
* @throws IOException if there's an exception while retrieving checksums.
|
||||
*/
|
||||
public static boolean checksumsAreEqual(FileSystem sourceFS, Path source,
|
||||
FileSystem targetFS, Path target)
|
||||
throws IOException {
|
||||
FileChecksum sourceChecksum = null;
|
||||
FileChecksum targetChecksum = null;
|
||||
try {
|
||||
sourceChecksum = sourceFS.getFileChecksum(source);
|
||||
targetChecksum = targetFS.getFileChecksum(target);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Unable to retrieve checksum for " + source + " or " + target, e);
|
||||
}
|
||||
return (sourceChecksum == null || targetChecksum == null ||
|
||||
sourceChecksum.equals(targetChecksum));
|
||||
}
|
||||
|
||||
/* see if two file systems are the same or not
|
||||
*
|
||||
*/
|
||||
public static boolean compareFs(FileSystem srcFs, FileSystem destFs) {
|
||||
URI srcUri = srcFs.getUri();
|
||||
URI dstUri = destFs.getUri();
|
||||
if (srcUri.getScheme() == null) {
|
||||
return false;
|
||||
}
|
||||
if (!srcUri.getScheme().equals(dstUri.getScheme())) {
|
||||
return false;
|
||||
}
|
||||
String srcHost = srcUri.getHost();
|
||||
String dstHost = dstUri.getHost();
|
||||
if ((srcHost != null) && (dstHost != null)) {
|
||||
try {
|
||||
srcHost = InetAddress.getByName(srcHost).getCanonicalHostName();
|
||||
dstHost = InetAddress.getByName(dstHost).getCanonicalHostName();
|
||||
} catch(UnknownHostException ue) {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug("Could not compare file-systems. Unknown host: ", ue);
|
||||
return false;
|
||||
}
|
||||
if (!srcHost.equals(dstHost)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (srcHost == null && dstHost != null) {
|
||||
return false;
|
||||
}
|
||||
else if (srcHost != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
//check for ports
|
||||
|
||||
return srcUri.getPort() == dstUri.getPort();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,106 @@
|
|||
/**
|
||||
* 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.hadoop.tools.util;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.io.retry.RetryPolicy;
|
||||
import org.apache.hadoop.io.retry.RetryPolicies;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* This class represents commands that be retried on failure, in a configurable
|
||||
* manner.
|
||||
*/
|
||||
public abstract class RetriableCommand {
|
||||
|
||||
private static Log LOG = LogFactory.getLog(RetriableCommand.class);
|
||||
|
||||
private static final long DELAY_MILLISECONDS = 500;
|
||||
private static final int MAX_RETRIES = 3;
|
||||
|
||||
private RetryPolicy retryPolicy = RetryPolicies.
|
||||
exponentialBackoffRetry(MAX_RETRIES, DELAY_MILLISECONDS, TimeUnit.MILLISECONDS);
|
||||
protected String description;
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
* @param description The human-readable description of the command.
|
||||
*/
|
||||
public RetriableCommand(String description) {
|
||||
this.description = description;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
* @param description The human-readable description of the command.
|
||||
* @param retryPolicy The RetryHandler to be used to compute retries.
|
||||
*/
|
||||
public RetriableCommand(String description, RetryPolicy retryPolicy) {
|
||||
this(description);
|
||||
setRetryPolicy(retryPolicy);
|
||||
}
|
||||
|
||||
/**
|
||||
* Implement this interface-method define the command-logic that will be
|
||||
* retried on failure (i.e. with Exception).
|
||||
* @param arguments Argument-list to the command.
|
||||
* @return Generic "Object".
|
||||
* @throws Exception Throws Exception on complete failure.
|
||||
*/
|
||||
protected abstract Object doExecute(Object... arguments) throws Exception;
|
||||
|
||||
/**
|
||||
* The execute() method invokes doExecute() until either:
|
||||
* 1. doExecute() succeeds, or
|
||||
* 2. the command may no longer be retried (e.g. runs out of retry-attempts).
|
||||
* @param arguments The list of arguments for the command.
|
||||
* @return Generic "Object" from doExecute(), on success.
|
||||
* @throws IOException, IOException, on complete failure.
|
||||
*/
|
||||
public Object execute(Object... arguments) throws Exception {
|
||||
Exception latestException;
|
||||
int counter = 0;
|
||||
do {
|
||||
try {
|
||||
return doExecute(arguments);
|
||||
} catch(Exception exception) {
|
||||
LOG.error("Failure in Retriable command: " + description, exception);
|
||||
latestException = exception;
|
||||
}
|
||||
counter++;
|
||||
} while (retryPolicy.shouldRetry(latestException, counter, 0, true).equals(RetryPolicy.RetryAction.RETRY));
|
||||
|
||||
throw new IOException("Couldn't run retriable-command: " + description,
|
||||
latestException);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fluent-interface to change the RetryHandler.
|
||||
* @param retryHandler The new RetryHandler instance to be used.
|
||||
* @return Self.
|
||||
*/
|
||||
public RetriableCommand setRetryPolicy(RetryPolicy retryHandler) {
|
||||
this.retryPolicy = retryHandler;
|
||||
return this;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,139 @@
|
|||
/**
|
||||
* 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.hadoop.tools.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
/**
|
||||
* The ThrottleInputStream provides bandwidth throttling on a specified
|
||||
* InputStream. It is implemented as a wrapper on top of another InputStream
|
||||
* instance.
|
||||
* The throttling works by examining the number of bytes read from the underlying
|
||||
* InputStream from the beginning, and sleep()ing for a time interval if
|
||||
* the byte-transfer is found exceed the specified tolerable maximum.
|
||||
* (Thus, while the read-rate might exceed the maximum for a given short interval,
|
||||
* the average tends towards the specified maximum, overall.)
|
||||
*/
|
||||
public class ThrottledInputStream extends InputStream {
|
||||
|
||||
private final InputStream rawStream;
|
||||
private final long maxBytesPerSec;
|
||||
private final long startTime = System.currentTimeMillis();
|
||||
|
||||
private long bytesRead = 0;
|
||||
private long totalSleepTime = 0;
|
||||
|
||||
private static final long SLEEP_DURATION_MS = 50;
|
||||
|
||||
public ThrottledInputStream(InputStream rawStream) {
|
||||
this(rawStream, Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
public ThrottledInputStream(InputStream rawStream, long maxBytesPerSec) {
|
||||
assert maxBytesPerSec > 0 : "Bandwidth " + maxBytesPerSec + " is invalid";
|
||||
this.rawStream = rawStream;
|
||||
this.maxBytesPerSec = maxBytesPerSec;
|
||||
}
|
||||
|
||||
/** @inheritDoc */
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
throttle();
|
||||
int data = rawStream.read();
|
||||
if (data != -1) {
|
||||
bytesRead++;
|
||||
}
|
||||
return data;
|
||||
}
|
||||
|
||||
/** @inheritDoc */
|
||||
@Override
|
||||
public int read(byte[] b) throws IOException {
|
||||
throttle();
|
||||
int readLen = rawStream.read(b);
|
||||
if (readLen != -1) {
|
||||
bytesRead += readLen;
|
||||
}
|
||||
return readLen;
|
||||
}
|
||||
|
||||
/** @inheritDoc */
|
||||
@Override
|
||||
public int read(byte[] b, int off, int len) throws IOException {
|
||||
throttle();
|
||||
int readLen = rawStream.read(b, off, len);
|
||||
if (readLen != -1) {
|
||||
bytesRead += readLen;
|
||||
}
|
||||
return readLen;
|
||||
}
|
||||
|
||||
private void throttle() throws IOException {
|
||||
if (getBytesPerSec() > maxBytesPerSec) {
|
||||
try {
|
||||
Thread.sleep(SLEEP_DURATION_MS);
|
||||
totalSleepTime += SLEEP_DURATION_MS;
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Thread aborted", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for the number of bytes read from this stream, since creation.
|
||||
* @return The number of bytes.
|
||||
*/
|
||||
public long getTotalBytesRead() {
|
||||
return bytesRead;
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for the read-rate from this stream, since creation.
|
||||
* Calculated as bytesRead/elapsedTimeSinceStart.
|
||||
* @return Read rate, in bytes/sec.
|
||||
*/
|
||||
public long getBytesPerSec() {
|
||||
long elapsed = (System.currentTimeMillis() - startTime) / 1000;
|
||||
if (elapsed == 0) {
|
||||
return bytesRead;
|
||||
} else {
|
||||
return bytesRead / elapsed;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter the total time spent in sleep.
|
||||
* @return Number of milliseconds spent in sleep.
|
||||
*/
|
||||
public long getTotalSleepTime() {
|
||||
return totalSleepTime;
|
||||
}
|
||||
|
||||
/** @inheritDoc */
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ThrottledInputStream{" +
|
||||
"bytesRead=" + bytesRead +
|
||||
", maxBytesPerSec=" + maxBytesPerSec +
|
||||
", bytesPerSec=" + getBytesPerSec() +
|
||||
", totalSleepTime=" + totalSleepTime +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
|
||||
<!-- Do not modify this file directly. Anything that need to be overwritten,
|
||||
need to be done so, through -D switches or customized conf -->
|
||||
|
||||
<configuration>
|
||||
|
||||
<property>
|
||||
<name>distcp.dynamic.strategy.impl</name>
|
||||
<value>org.apache.hadoop.tools.mapred.lib.DynamicInputFormat</value>
|
||||
<description>Implementation of dynamic input format</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>distcp.static.strategy.impl</name>
|
||||
<value>org.apache.hadoop.tools.mapred.UniformSizeInputFormat</value>
|
||||
<description>Implementation of static input format</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>mapred.job.map.memory.mb</name>
|
||||
<value>1024</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>mapred.job.reduce.memory.mb</name>
|
||||
<value>1024</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>mapred.reducer.new-api</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>mapreduce.reduce.class</name>
|
||||
<value>org.apache.hadoop.mapreduce.Reducer</value>
|
||||
</property>
|
||||
|
||||
</configuration>
|
|
@ -0,0 +1,98 @@
|
|||
<?xml version="1.0" encoding="ISO-8859-1" ?>
|
||||
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<faqs xmlns="http://maven.apache.org/FML/1.0.1"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/FML/1.0.1 http://maven.apache.org/xsd/fml-1.0.1.xsd"
|
||||
title="Frequently Asked Questions">
|
||||
<part id="General">
|
||||
<title>General</title>
|
||||
|
||||
<faq id="Update">
|
||||
<question>Why does -update not create the parent source-directory under
|
||||
a pre-existing target directory?</question>
|
||||
<answer>The behaviour of <code>-update</code> and <code>-overwrite</code>
|
||||
is described in detail in the Usage section of this document. In short,
|
||||
if either option is used with a pre-existing destination directory, the
|
||||
<strong>contents</strong> of each source directory is copied over, rather
|
||||
than the source-directory itself.
|
||||
This behaviour is consistent with the legacy DistCp implementation as well.
|
||||
</answer>
|
||||
</faq>
|
||||
|
||||
<faq id="Deviation">
|
||||
<question>How does the new DistCp differ in semantics from the Legacy
|
||||
DistCp?</question>
|
||||
<answer>
|
||||
<ul>
|
||||
<li>Files that are skipped during copy used to also have their
|
||||
file-attributes (permissions, owner/group info, etc.) unchanged,
|
||||
when copied with Legacy DistCp. These are now updated, even if
|
||||
the file-copy is skipped.</li>
|
||||
<li>Empty root directories among the source-path inputs were not
|
||||
created at the target, in Legacy DistCp. These are now created.</li>
|
||||
</ul>
|
||||
</answer>
|
||||
</faq>
|
||||
|
||||
<faq id="nMaps">
|
||||
<question>Why does the new DistCp use more maps than legacy DistCp?</question>
|
||||
<answer>
|
||||
<p>Legacy DistCp works by figuring out what files need to be actually
|
||||
copied to target <strong>before</strong> the copy-job is launched, and then
|
||||
launching as many maps as required for copy. So if a majority of the files
|
||||
need to be skipped (because they already exist, for example), fewer maps
|
||||
will be needed. As a consequence, the time spent in setup (i.e. before the
|
||||
M/R job) is higher.</p>
|
||||
<p>The new DistCp calculates only the contents of the source-paths. It
|
||||
doesn't try to filter out what files can be skipped. That decision is put-
|
||||
off till the M/R job runs. This is much faster (vis-a-vis execution-time),
|
||||
but the number of maps launched will be as specified in the <code>-m</code>
|
||||
option, or 20 (default) if unspecified.</p>
|
||||
</answer>
|
||||
</faq>
|
||||
|
||||
<faq id="more_maps">
|
||||
<question>Why does DistCp not run faster when more maps are specified?</question>
|
||||
<answer>
|
||||
<p>At present, the smallest unit of work for DistCp is a file. i.e.,
|
||||
a file is processed by only one map. Increasing the number of maps to
|
||||
a value exceeding the number of files would yield no performance
|
||||
benefit. The number of maps lauched would equal the number of files.</p>
|
||||
</answer>
|
||||
</faq>
|
||||
|
||||
<faq id="client_mem">
|
||||
<question>Why does DistCp run out of memory?</question>
|
||||
<answer>
|
||||
<p>If the number of individual files/directories being copied from
|
||||
the source path(s) is extremely large (e.g. 1,000,000 paths), DistCp might
|
||||
run out of memory while determining the list of paths for copy. This is
|
||||
not unique to the new DistCp implementation.</p>
|
||||
<p>To get around this, consider changing the <code>-Xmx</code> JVM
|
||||
heap-size parameters, as follows:</p>
|
||||
<p><code>bash$ export HADOOP_CLIENT_OPTS="-Xms64m -Xmx1024m"</code></p>
|
||||
<p><code>bash$ hadoop distcp /source /target</code></p>
|
||||
</answer>
|
||||
</faq>
|
||||
|
||||
</part>
|
||||
</faqs>
|
|
@ -0,0 +1,47 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<!-- START SNIPPET: docDescriptor -->
|
||||
<document xmlns="http://maven.apache.org/DOCUMENT/1.0.1"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/DOCUMENT/1.0.1 http://maven.apache.org/xsd/document-1.0.1.xsd"
|
||||
outputName="distcp">
|
||||
|
||||
<meta>
|
||||
<title>${project.name}</title>
|
||||
</meta>
|
||||
|
||||
<toc name="Table of Contents">
|
||||
<item name="Introduction" ref="index.xml"/>
|
||||
<item name="Usage" ref="usage.xml"/>
|
||||
<item name="Command Line Reference" ref="cli.xml"/>
|
||||
<item name="Architecture" ref="architecture.xml"/>
|
||||
<item name="Appendix" ref="appendix.xml"/>
|
||||
<item name="FAQ" ref="faq.fml"/>
|
||||
</toc>
|
||||
<cover>
|
||||
<coverTitle>${project.name}</coverTitle>
|
||||
<coverSubTitle>v. ${project.version}</coverSubTitle>
|
||||
<coverType>User Guide</coverType>
|
||||
<projectName>${project.name}</projectName>
|
||||
<companyName>Apache Hadoop</companyName>
|
||||
</cover>
|
||||
</document>
|
|
@ -0,0 +1,125 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<document xmlns="http://maven.apache.org/XDOC/2.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
|
||||
<head>
|
||||
<title>Appendix</title>
|
||||
</head>
|
||||
<body>
|
||||
<section name="Map sizing">
|
||||
|
||||
<p> By default, DistCp makes an attempt to size each map comparably so
|
||||
that each copies roughly the same number of bytes. Note that files are the
|
||||
finest level of granularity, so increasing the number of simultaneous
|
||||
copiers (i.e. maps) may not always increase the number of
|
||||
simultaneous copies nor the overall throughput.</p>
|
||||
|
||||
<p> The new DistCp also provides a strategy to "dynamically" size maps,
|
||||
allowing faster data-nodes to copy more bytes than slower nodes. Using
|
||||
<code>-strategy dynamic</code> (explained in the Architecture), rather
|
||||
than to assign a fixed set of source-files to each map-task, files are
|
||||
instead split into several sets. The number of sets exceeds the number of
|
||||
maps, usually by a factor of 2-3. Each map picks up and copies all files
|
||||
listed in a chunk. When a chunk is exhausted, a new chunk is acquired and
|
||||
processed, until no more chunks remain.</p>
|
||||
|
||||
<p> By not assigning a source-path to a fixed map, faster map-tasks (i.e.
|
||||
data-nodes) are able to consume more chunks, and thus copy more data,
|
||||
than slower nodes. While this distribution isn't uniform, it is
|
||||
<strong>fair</strong> with regard to each mapper's capacity.</p>
|
||||
|
||||
<p>The dynamic-strategy is implemented by the DynamicInputFormat. It
|
||||
provides superior performance under most conditions. </p>
|
||||
|
||||
<p>Tuning the number of maps to the size of the source and
|
||||
destination clusters, the size of the copy, and the available
|
||||
bandwidth is recommended for long-running and regularly run jobs.</p>
|
||||
|
||||
</section>
|
||||
|
||||
<section name="Copying between versions of HDFS">
|
||||
|
||||
<p>For copying between two different versions of Hadoop, one will
|
||||
usually use HftpFileSystem. This is a read-only FileSystem, so DistCp
|
||||
must be run on the destination cluster (more specifically, on
|
||||
TaskTrackers that can write to the destination cluster). Each source is
|
||||
specified as <code>hftp://<dfs.http.address>/<path></code>
|
||||
(the default <code>dfs.http.address</code> is
|
||||
<namenode>:50070).</p>
|
||||
|
||||
</section>
|
||||
|
||||
<section name="Map/Reduce and other side-effects">
|
||||
|
||||
<p>As has been mentioned in the preceding, should a map fail to copy
|
||||
one of its inputs, there will be several side-effects.</p>
|
||||
|
||||
<ul>
|
||||
|
||||
<li>Unless <code>-overwrite</code> is specified, files successfully
|
||||
copied by a previous map on a re-execution will be marked as
|
||||
"skipped".</li>
|
||||
|
||||
<li>If a map fails <code>mapred.map.max.attempts</code> times, the
|
||||
remaining map tasks will be killed (unless <code>-i</code> is
|
||||
set).</li>
|
||||
|
||||
<li>If <code>mapred.speculative.execution</code> is set set
|
||||
<code>final</code> and <code>true</code>, the result of the copy is
|
||||
undefined.</li>
|
||||
|
||||
</ul>
|
||||
|
||||
</section>
|
||||
|
||||
<section name="SSL Configurations for HSFTP sources:">
|
||||
|
||||
<p>To use an HSFTP source (i.e. using the hsftp protocol), a Map-Red SSL
|
||||
configuration file needs to be specified (via the <code>-mapredSslConf</code>
|
||||
option). This must specify 3 parameters:</p>
|
||||
|
||||
<ul>
|
||||
<li><code>ssl.client.truststore.location</code>: The local-filesystem
|
||||
location of the trust-store file, containing the certificate for
|
||||
the namenode.</li>
|
||||
|
||||
<li><code>ssl.client.truststore.type</code>: (Optional) The format of
|
||||
the trust-store file.</li>
|
||||
|
||||
<li><code>ssl.client.truststore.password</code>: (Optional) Password
|
||||
for the trust-store file.</li>
|
||||
|
||||
</ul>
|
||||
|
||||
<p>The following is an example of the contents of the contents of
|
||||
a Map-Red SSL Configuration file:</p>
|
||||
|
||||
<p> <br/> <code> <configuration> </code> </p>
|
||||
|
||||
<p> <br/> <code><property> </code> </p>
|
||||
<p> <code><name>ssl.client.truststore.location</name> </code> </p>
|
||||
<p> <code><value>/work/keystore.jks</value> </code> </p>
|
||||
<p> <code><description>Truststore to be used by clients like distcp. Must be specified. </description></code> </p>
|
||||
<p> <br/> <code></property> </code> </p>
|
||||
|
||||
<p><code> <property> </code> </p>
|
||||
<p> <code><name>ssl.client.truststore.password</name> </code> </p>
|
||||
<p> <code><value>changeme</value> </code> </p>
|
||||
<p> <code><description>Optional. Default value is "". </description> </code> </p>
|
||||
<p> <code></property> </code> </p>
|
||||
|
||||
<p> <br/> <code> <property> </code> </p>
|
||||
<p> <code> <name>ssl.client.truststore.type</name></code> </p>
|
||||
<p> <code> <value>jks</value></code> </p>
|
||||
<p> <code> <description>Optional. Default value is "jks". </description></code> </p>
|
||||
<p> <code> </property> </code> </p>
|
||||
|
||||
<p> <code> <br/> </configuration> </code> </p>
|
||||
|
||||
<p><br/>The SSL configuration file must be in the class-path of the
|
||||
DistCp program.</p>
|
||||
|
||||
</section>
|
||||
|
||||
</body>
|
||||
</document>
|
|
@ -0,0 +1,200 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<document xmlns="http://maven.apache.org/XDOC/2.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
|
||||
<head>
|
||||
<title>Architecture of DistCp</title>
|
||||
</head>
|
||||
<body>
|
||||
<section name="Architecture">
|
||||
|
||||
<p>The components of the new DistCp may be classified into the following
|
||||
categories: </p>
|
||||
|
||||
<ul>
|
||||
|
||||
<li>DistCp Driver</li>
|
||||
<li>Copy-listing generator</li>
|
||||
<li>Input-formats and Map-Reduce components</li>
|
||||
|
||||
</ul>
|
||||
|
||||
<subsection name="DistCp Driver">
|
||||
<p>The DistCp Driver components are responsible for:</p>
|
||||
|
||||
<ul>
|
||||
<li>Parsing the arguments passed to the DistCp command on the
|
||||
command-line, via:
|
||||
<ul>
|
||||
<li>OptionsParser, and</li>
|
||||
<li>DistCpOptionsSwitch</li>
|
||||
</ul>
|
||||
</li>
|
||||
<li>Assembling the command arguments into an appropriate
|
||||
DistCpOptions object, and initializing DistCp. These arguments
|
||||
include:
|
||||
<ul>
|
||||
<li>Source-paths</li>
|
||||
<li>Target location</li>
|
||||
<li>Copy options (e.g. whether to update-copy, overwrite, which
|
||||
file-attributes to preserve, etc.)</li>
|
||||
</ul>
|
||||
</li>
|
||||
<li>Orchestrating the copy operation by:
|
||||
<ul>
|
||||
<li>Invoking the copy-listing-generator to create the list of
|
||||
files to be copied.</li>
|
||||
<li>Setting up and launching the Hadoop Map-Reduce Job to carry
|
||||
out the copy.</li>
|
||||
<li>Based on the options, either returning a handle to the
|
||||
Hadoop MR Job immediately, or waiting till completion.</li>
|
||||
</ul>
|
||||
</li>
|
||||
</ul>
|
||||
<br/>
|
||||
|
||||
<p>The parser-elements are exercised only from the command-line (or if
|
||||
DistCp::run() is invoked). The DistCp class may also be used
|
||||
programmatically, by constructing the DistCpOptions object, and
|
||||
initializing a DistCp object appropriately.</p>
|
||||
|
||||
</subsection>
|
||||
|
||||
<subsection name="Copy-listing generator">
|
||||
|
||||
<p>The copy-listing-generator classes are responsible for creating the
|
||||
list of files/directories to be copied from source. They examine
|
||||
the contents of the source-paths (files/directories, including
|
||||
wild-cards), and record all paths that need copy into a sequence-
|
||||
file, for consumption by the DistCp Hadoop Job. The main classes in
|
||||
this module include:</p>
|
||||
|
||||
<ol>
|
||||
|
||||
<li>CopyListing: The interface that should be implemented by any
|
||||
copy-listing-generator implementation. Also provides the factory
|
||||
method by which the concrete CopyListing implementation is
|
||||
chosen.</li>
|
||||
|
||||
<li>SimpleCopyListing: An implementation of CopyListing that accepts
|
||||
multiple source paths (files/directories), and recursively lists
|
||||
all the individual files and directories under each, for
|
||||
copy.</li>
|
||||
|
||||
<li>GlobbedCopyListing: Another implementation of CopyListing that
|
||||
expands wild-cards in the source paths.</li>
|
||||
|
||||
<li>FileBasedCopyListing: An implementation of CopyListing that
|
||||
reads the source-path list from a specified file.</li>
|
||||
|
||||
</ol>
|
||||
<p/>
|
||||
|
||||
<p>Based on whether a source-file-list is specified in the
|
||||
DistCpOptions, the source-listing is generated in one of the
|
||||
following ways:</p>
|
||||
|
||||
<ol>
|
||||
|
||||
<li>If there's no source-file-list, the GlobbedCopyListing is used.
|
||||
All wild-cards are expanded, and all the expansions are
|
||||
forwarded to the SimpleCopyListing, which in turn constructs the
|
||||
listing (via recursive descent of each path). </li>
|
||||
|
||||
<li>If a source-file-list is specified, the FileBasedCopyListing is
|
||||
used. Source-paths are read from the specified file, and then
|
||||
forwarded to the GlobbedCopyListing. The listing is then
|
||||
constructed as described above.</li>
|
||||
|
||||
</ol>
|
||||
|
||||
<br/>
|
||||
|
||||
<p>One may customize the method by which the copy-listing is
|
||||
constructed by providing a custom implementation of the CopyListing
|
||||
interface. The behaviour of DistCp differs here from the legacy
|
||||
DistCp, in how paths are considered for copy. </p>
|
||||
|
||||
<p>The legacy implementation only lists those paths that must
|
||||
definitely be copied on to target.
|
||||
E.g. if a file already exists at the target (and -overwrite isn't
|
||||
specified), the file isn't even considered in the Map-Reduce Copy
|
||||
Job. Determining this during setup (i.e. before the Map-Reduce Job)
|
||||
involves file-size and checksum-comparisons that are potentially
|
||||
time-consuming.</p>
|
||||
|
||||
<p>The new DistCp postpones such checks until the Map-Reduce Job, thus
|
||||
reducing setup time. Performance is enhanced further since these
|
||||
checks are parallelized across multiple maps.</p>
|
||||
|
||||
</subsection>
|
||||
|
||||
<subsection name="Input-formats and Map-Reduce components">
|
||||
|
||||
<p> The Input-formats and Map-Reduce components are responsible for
|
||||
the actual copy of files and directories from the source to the
|
||||
destination path. The listing-file created during copy-listing
|
||||
generation is consumed at this point, when the copy is carried
|
||||
out. The classes of interest here include:</p>
|
||||
|
||||
<ul>
|
||||
<li><strong>UniformSizeInputFormat:</strong> This implementation of
|
||||
org.apache.hadoop.mapreduce.InputFormat provides equivalence
|
||||
with Legacy DistCp in balancing load across maps.
|
||||
The aim of the UniformSizeInputFormat is to make each map copy
|
||||
roughly the same number of bytes. Apropos, the listing file is
|
||||
split into groups of paths, such that the sum of file-sizes in
|
||||
each InputSplit is nearly equal to every other map. The splitting
|
||||
isn't always perfect, but its trivial implementation keeps the
|
||||
setup-time low.</li>
|
||||
|
||||
<li><strong>DynamicInputFormat and DynamicRecordReader:</strong>
|
||||
<p> The DynamicInputFormat implements org.apache.hadoop.mapreduce.InputFormat,
|
||||
and is new to DistCp. The listing-file is split into several
|
||||
"chunk-files", the exact number of chunk-files being a multiple
|
||||
of the number of maps requested for in the Hadoop Job. Each map
|
||||
task is "assigned" one of the chunk-files (by renaming the chunk
|
||||
to the task's id), before the Job is launched.</p>
|
||||
|
||||
<p>Paths are read from each chunk using the DynamicRecordReader,
|
||||
and processed in the CopyMapper. After all the paths in a chunk
|
||||
are processed, the current chunk is deleted and a new chunk is
|
||||
acquired. The process continues until no more chunks are
|
||||
available.</p>
|
||||
<p>This "dynamic" approach allows faster map-tasks to consume
|
||||
more paths than slower ones, thus speeding up the DistCp job
|
||||
overall. </p>
|
||||
</li>
|
||||
|
||||
<li><strong>CopyMapper:</strong> This class implements the physical
|
||||
file-copy. The input-paths are checked against the input-options
|
||||
(specified in the Job's Configuration), to determine whether a
|
||||
file needs copy. A file will be copied only if at least one of
|
||||
the following is true:
|
||||
<ul>
|
||||
<li>A file with the same name doesn't exist at target.</li>
|
||||
<li>A file with the same name exists at target, but has a
|
||||
different file size.</li>
|
||||
<li>A file with the same name exists at target, but has a
|
||||
different checksum, and -skipcrccheck isn't mentioned.</li>
|
||||
<li>A file with the same name exists at target, but -overwrite
|
||||
is specified.</li>
|
||||
<li>A file with the same name exists at target, but differs in
|
||||
block-size (and block-size needs to be preserved.</li>
|
||||
</ul>
|
||||
</li>
|
||||
|
||||
<li><strong>CopyCommitter:</strong>
|
||||
This class is responsible for the commit-phase of the DistCp
|
||||
job, including:
|
||||
<ul>
|
||||
<li>Preservation of directory-permissions (if specified in the
|
||||
options)</li>
|
||||
<li>Clean-up of temporary-files, work-directories, etc.</li>
|
||||
</ul>
|
||||
</li>
|
||||
</ul>
|
||||
</subsection>
|
||||
</section>
|
||||
</body>
|
||||
</document>
|
|
@ -0,0 +1,123 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<document xmlns="http://maven.apache.org/XDOC/2.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
|
||||
<head>
|
||||
<title>Command Line Options</title>
|
||||
</head>
|
||||
<body>
|
||||
<section name="Options Index">
|
||||
<table>
|
||||
<tr><th> Flag </th><th> Description </th><th> Notes </th></tr>
|
||||
|
||||
<tr><td><code>-p[rbugp]</code></td>
|
||||
<td>Preserve<br/>
|
||||
r: replication number<br/>
|
||||
b: block size<br/>
|
||||
u: user<br/>
|
||||
g: group<br/>
|
||||
p: permission<br/></td>
|
||||
<td>Modification times are not preserved. Also, when
|
||||
<code>-update</code> is specified, status updates will
|
||||
<strong>not</strong> be synchronized unless the file sizes
|
||||
also differ (i.e. unless the file is re-created).
|
||||
</td></tr>
|
||||
<tr><td><code>-i</code></td>
|
||||
<td>Ignore failures</td>
|
||||
<td>As explained in the Appendix, this option
|
||||
will keep more accurate statistics about the copy than the
|
||||
default case. It also preserves logs from failed copies, which
|
||||
can be valuable for debugging. Finally, a failing map will not
|
||||
cause the job to fail before all splits are attempted.
|
||||
</td></tr>
|
||||
<tr><td><code>-log <logdir></code></td>
|
||||
<td>Write logs to <logdir></td>
|
||||
<td>DistCp keeps logs of each file it attempts to copy as map
|
||||
output. If a map fails, the log output will not be retained if
|
||||
it is re-executed.
|
||||
</td></tr>
|
||||
<tr><td><code>-m <num_maps></code></td>
|
||||
<td>Maximum number of simultaneous copies</td>
|
||||
<td>Specify the number of maps to copy data. Note that more maps
|
||||
may not necessarily improve throughput.
|
||||
</td></tr>
|
||||
<tr><td><code>-overwrite</code></td>
|
||||
<td>Overwrite destination</td>
|
||||
<td>If a map fails and <code>-i</code> is not specified, all the
|
||||
files in the split, not only those that failed, will be recopied.
|
||||
As discussed in the Usage documentation, it also changes
|
||||
the semantics for generating destination paths, so users should
|
||||
use this carefully.
|
||||
</td></tr>
|
||||
<tr><td><code>-update</code></td>
|
||||
<td>Overwrite if src size different from dst size</td>
|
||||
<td>As noted in the preceding, this is not a "sync"
|
||||
operation. The only criterion examined is the source and
|
||||
destination file sizes; if they differ, the source file
|
||||
replaces the destination file. As discussed in the
|
||||
Usage documentation, it also changes the semantics for
|
||||
generating destination paths, so users should use this carefully.
|
||||
</td></tr>
|
||||
<tr><td><code>-f <urilist_uri></code></td>
|
||||
<td>Use list at <urilist_uri> as src list</td>
|
||||
<td>This is equivalent to listing each source on the command
|
||||
line. The <code>urilist_uri</code> list should be a fully
|
||||
qualified URI.
|
||||
</td></tr>
|
||||
<tr><td><code>-filelimit <n></code></td>
|
||||
<td>Limit the total number of files to be <= n</td>
|
||||
<td><strong>Deprecated!</strong> Ignored in the new DistCp.
|
||||
</td></tr>
|
||||
<tr><td><code>-sizelimit <n></code></td>
|
||||
<td>Limit the total size to be <= n bytes</td>
|
||||
<td><strong>Deprecated!</strong> Ignored in the new DistCp.
|
||||
</td></tr>
|
||||
<tr><td><code>-delete</code></td>
|
||||
<td>Delete the files existing in the dst but not in src</td>
|
||||
<td>The deletion is done by FS Shell. So the trash will be used,
|
||||
if it is enable.
|
||||
</td></tr>
|
||||
<tr><td><code>-strategy {dynamic|uniformsize}</code></td>
|
||||
<td>Choose the copy-strategy to be used in DistCp.</td>
|
||||
<td>By default, uniformsize is used. (i.e. Maps are balanced on the
|
||||
total size of files copied by each map. Similar to legacy.)
|
||||
If "dynamic" is specified, <code>DynamicInputFormat</code> is
|
||||
used instead. (This is described in the Architecture section,
|
||||
under InputFormats.)
|
||||
</td></tr>
|
||||
<tr><td><code>-bandwidth</code></td>
|
||||
<td>Specify bandwidth per map, in MB/second.</td>
|
||||
<td>Each map will be restricted to consume only the specified
|
||||
bandwidth. This is not always exact. The map throttles back
|
||||
its bandwidth consumption during a copy, such that the
|
||||
<strong>net</strong> bandwidth used tends towards the
|
||||
specified value.
|
||||
</td></tr>
|
||||
<tr><td><code>-atomic {-tmp <tmp_dir>}</code></td>
|
||||
<td>Specify atomic commit, with optional tmp directory.</td>
|
||||
<td><code>-atomic</code> instructs DistCp to copy the source
|
||||
data to a temporary target location, and then move the
|
||||
temporary target to the final-location atomically. Data will
|
||||
either be available at final target in a complete and consistent
|
||||
form, or not at all.
|
||||
Optionally, <code>-tmp</code> may be used to specify the
|
||||
location of the tmp-target. If not specified, a default is
|
||||
chosen. <strong>Note:</strong> tmp_dir must be on the final
|
||||
target cluster.
|
||||
</td></tr>
|
||||
<tr><td><code>-mapredSslConf <ssl_conf_file></code></td>
|
||||
<td>Specify SSL Config file, to be used with HSFTP source</td>
|
||||
<td>When using the hsftp protocol with a source, the security-
|
||||
related properties may be specified in a config-file and
|
||||
passed to DistCp. <ssl_conf_file> needs to be in
|
||||
the classpath.
|
||||
</td></tr>
|
||||
<tr><td><code>-async</code></td>
|
||||
<td>Run DistCp asynchronously. Quits as soon as the Hadoop
|
||||
Job is launched.</td>
|
||||
<td>The Hadoop Job-id is logged, for tracking.
|
||||
</td></tr>
|
||||
</table>
|
||||
</section>
|
||||
</body>
|
||||
</document>
|
|
@ -0,0 +1,32 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<document xmlns="http://maven.apache.org/XDOC/2.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
|
||||
<head>
|
||||
<title>DistCp</title>
|
||||
</head>
|
||||
<body>
|
||||
<section name="Overview">
|
||||
<p>
|
||||
DistCp (distributed copy) is a tool used for large inter/intra-cluster
|
||||
copying. It uses Map/Reduce to effect its distribution, error
|
||||
handling and recovery, and reporting. It expands a list of files and
|
||||
directories into input to map tasks, each of which will copy a partition
|
||||
of the files specified in the source list.
|
||||
</p>
|
||||
<p>
|
||||
The erstwhile implementation of DistCp has its share of quirks and
|
||||
drawbacks, both in its usage, as well as its extensibility and
|
||||
performance. The purpose of the DistCp refactor was to fix these shortcomings,
|
||||
enabling it to be used and extended programmatically. New paradigms have
|
||||
been introduced to improve runtime and setup performance, while simultaneously
|
||||
retaining the legacy behaviour as default.
|
||||
</p>
|
||||
<p>
|
||||
This document aims to describe the design of the new DistCp, its spanking
|
||||
new features, their optimal use, and any deviance from the legacy
|
||||
implementation.
|
||||
</p>
|
||||
</section>
|
||||
</body>
|
||||
</document>
|
|
@ -0,0 +1,147 @@
|
|||
<document xmlns="http://maven.apache.org/XDOC/2.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
|
||||
<head>
|
||||
<title>Usage </title>
|
||||
</head>
|
||||
<body>
|
||||
<section name="Basic Usage">
|
||||
<p>The most common invocation of DistCp is an inter-cluster copy:</p>
|
||||
<p><code>bash$ hadoop jar hadoop-distcp.jar hdfs://nn1:8020/foo/bar \</code><br/>
|
||||
<code> hdfs://nn2:8020/bar/foo</code></p>
|
||||
|
||||
<p>This will expand the namespace under <code>/foo/bar</code> on nn1
|
||||
into a temporary file, partition its contents among a set of map
|
||||
tasks, and start a copy on each TaskTracker from nn1 to nn2.</p>
|
||||
|
||||
<p>One can also specify multiple source directories on the command
|
||||
line:</p>
|
||||
<p><code>bash$ hadoop jar hadoop-distcp.jar hdfs://nn1:8020/foo/a \</code><br/>
|
||||
<code> hdfs://nn1:8020/foo/b \</code><br/>
|
||||
<code> hdfs://nn2:8020/bar/foo</code></p>
|
||||
|
||||
<p>Or, equivalently, from a file using the <code>-f</code> option:<br/>
|
||||
<code>bash$ hadoop jar hadoop-distcp.jar -f hdfs://nn1:8020/srclist \</code><br/>
|
||||
<code> hdfs://nn2:8020/bar/foo</code><br/></p>
|
||||
|
||||
<p>Where <code>srclist</code> contains<br/>
|
||||
<code>hdfs://nn1:8020/foo/a</code><br/>
|
||||
<code>hdfs://nn1:8020/foo/b</code></p>
|
||||
|
||||
<p>When copying from multiple sources, DistCp will abort the copy with
|
||||
an error message if two sources collide, but collisions at the
|
||||
destination are resolved per the <a href="#options">options</a>
|
||||
specified. By default, files already existing at the destination are
|
||||
skipped (i.e. not replaced by the source file). A count of skipped
|
||||
files is reported at the end of each job, but it may be inaccurate if a
|
||||
copier failed for some subset of its files, but succeeded on a later
|
||||
attempt.</p>
|
||||
|
||||
<p>It is important that each TaskTracker can reach and communicate with
|
||||
both the source and destination file systems. For HDFS, both the source
|
||||
and destination must be running the same version of the protocol or use
|
||||
a backwards-compatible protocol (see <a href="#cpver">Copying Between
|
||||
Versions</a>).</p>
|
||||
|
||||
<p>After a copy, it is recommended that one generates and cross-checks
|
||||
a listing of the source and destination to verify that the copy was
|
||||
truly successful. Since DistCp employs both Map/Reduce and the
|
||||
FileSystem API, issues in or between any of the three could adversely
|
||||
and silently affect the copy. Some have had success running with
|
||||
<code>-update</code> enabled to perform a second pass, but users should
|
||||
be acquainted with its semantics before attempting this.</p>
|
||||
|
||||
<p>It's also worth noting that if another client is still writing to a
|
||||
source file, the copy will likely fail. Attempting to overwrite a file
|
||||
being written at the destination should also fail on HDFS. If a source
|
||||
file is (re)moved before it is copied, the copy will fail with a
|
||||
FileNotFoundException.</p>
|
||||
|
||||
<p>Please refer to the detailed Command Line Reference for information
|
||||
on all the options available in DistCp.</p>
|
||||
|
||||
</section>
|
||||
<section name="Update and Overwrite">
|
||||
|
||||
<p><code>-update</code> is used to copy files from source that don't
|
||||
exist at the target, or have different contents. <code>-overwrite</code>
|
||||
overwrites target-files even if they exist at the source, or have the
|
||||
same contents.</p>
|
||||
|
||||
<p><br/>Update and Overwrite options warrant special attention, since their
|
||||
handling of source-paths varies from the defaults in a very subtle manner.
|
||||
Consider a copy from <code>/source/first/</code> and
|
||||
<code>/source/second/</code> to <code>/target/</code>, where the source
|
||||
paths have the following contents:</p>
|
||||
|
||||
<p><code>hdfs://nn1:8020/source/first/1</code><br/>
|
||||
<code>hdfs://nn1:8020/source/first/2</code><br/>
|
||||
<code>hdfs://nn1:8020/source/second/10</code><br/>
|
||||
<code>hdfs://nn1:8020/source/second/20</code><br/></p>
|
||||
|
||||
<p><br/>When DistCp is invoked without <code>-update</code> or
|
||||
<code>-overwrite</code>, the DistCp defaults would create directories
|
||||
<code>first/</code> and <code>second/</code>, under <code>/target</code>.
|
||||
Thus:<br/></p>
|
||||
|
||||
<p><code>distcp hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target</code></p>
|
||||
<p><br/>would yield the following contents in <code>/target</code>: </p>
|
||||
|
||||
<p><code>hdfs://nn2:8020/target/first/1</code><br/>
|
||||
<code>hdfs://nn2:8020/target/first/2</code><br/>
|
||||
<code>hdfs://nn2:8020/target/second/10</code><br/>
|
||||
<code>hdfs://nn2:8020/target/second/20</code><br/></p>
|
||||
|
||||
<p><br/>When either <code>-update</code> or <code>-overwrite</code> is
|
||||
specified, the <strong>contents</strong> of the source-directories
|
||||
are copied to target, and not the source directories themselves. Thus: </p>
|
||||
|
||||
<p><code>distcp -update hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target</code></p>
|
||||
|
||||
<p><br/>would yield the following contents in <code>/target</code>: </p>
|
||||
|
||||
<p><code>hdfs://nn2:8020/target/1</code><br/>
|
||||
<code>hdfs://nn2:8020/target/2</code><br/>
|
||||
<code>hdfs://nn2:8020/target/10</code><br/>
|
||||
<code>hdfs://nn2:8020/target/20</code><br/></p>
|
||||
|
||||
<p><br/>By extension, if both source folders contained a file with the same
|
||||
name (say, <code>0</code>), then both sources would map an entry to
|
||||
<code>/target/0</code> at the destination. Rather than to permit this
|
||||
conflict, DistCp will abort.</p>
|
||||
|
||||
<p><br/>Now, consider the following copy operation:</p>
|
||||
|
||||
<p><code>distcp hdfs://nn1:8020/source/first hdfs://nn1:8020/source/second hdfs://nn2:8020/target</code></p>
|
||||
|
||||
<p><br/>With sources/sizes:</p>
|
||||
|
||||
<p><code>hdfs://nn1:8020/source/first/1 32</code><br/>
|
||||
<code>hdfs://nn1:8020/source/first/2 32</code><br/>
|
||||
<code>hdfs://nn1:8020/source/second/10 64</code><br/>
|
||||
<code>hdfs://nn1:8020/source/second/20 32</code><br/></p>
|
||||
|
||||
<p><br/>And destination/sizes:</p>
|
||||
|
||||
<p><code>hdfs://nn2:8020/target/1 32</code><br/>
|
||||
<code>hdfs://nn2:8020/target/10 32</code><br/>
|
||||
<code>hdfs://nn2:8020/target/20 64</code><br/></p>
|
||||
|
||||
<p><br/>Will effect: </p>
|
||||
|
||||
<p><code>hdfs://nn2:8020/target/1 32</code><br/>
|
||||
<code>hdfs://nn2:8020/target/2 32</code><br/>
|
||||
<code>hdfs://nn2:8020/target/10 64</code><br/>
|
||||
<code>hdfs://nn2:8020/target/20 32</code><br/></p>
|
||||
|
||||
<p><br/><code>1</code> is skipped because the file-length and contents match.
|
||||
<code>2</code> is copied because it doesn't exist at the target.
|
||||
<code>10</code> and <code>20</code> are overwritten since the contents
|
||||
don't match the source. </p>
|
||||
|
||||
<p>If <code>-update</code> is used, <code>1</code> is overwritten as well.</p>
|
||||
|
||||
</section>
|
||||
</body>
|
||||
|
||||
</document>
|
|
@ -0,0 +1,139 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.task.MapContextImpl;
|
||||
import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
|
||||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.io.IOException;
|
||||
|
||||
public class StubContext {
|
||||
|
||||
private StubStatusReporter reporter = new StubStatusReporter();
|
||||
private RecordReader<Text, FileStatus> reader;
|
||||
private StubInMemoryWriter writer = new StubInMemoryWriter();
|
||||
private Mapper<Text, FileStatus, Text, Text>.Context mapperContext;
|
||||
|
||||
public StubContext(Configuration conf, RecordReader<Text, FileStatus> reader,
|
||||
int taskId) throws IOException, InterruptedException {
|
||||
|
||||
WrappedMapper<Text, FileStatus, Text, Text> wrappedMapper
|
||||
= new WrappedMapper<Text, FileStatus, Text, Text>();
|
||||
|
||||
MapContextImpl<Text, FileStatus, Text, Text> contextImpl
|
||||
= new MapContextImpl<Text, FileStatus, Text, Text>(conf,
|
||||
getTaskAttemptID(taskId), reader, writer,
|
||||
null, reporter, null);
|
||||
|
||||
this.reader = reader;
|
||||
this.mapperContext = wrappedMapper.getMapContext(contextImpl);
|
||||
}
|
||||
|
||||
public Mapper<Text, FileStatus, Text, Text>.Context getContext() {
|
||||
return mapperContext;
|
||||
}
|
||||
|
||||
public StatusReporter getReporter() {
|
||||
return reporter;
|
||||
}
|
||||
|
||||
public RecordReader<Text, FileStatus> getReader() {
|
||||
return reader;
|
||||
}
|
||||
|
||||
public StubInMemoryWriter getWriter() {
|
||||
return writer;
|
||||
}
|
||||
|
||||
public static class StubStatusReporter extends StatusReporter {
|
||||
|
||||
private Counters counters = new Counters();
|
||||
|
||||
public StubStatusReporter() {
|
||||
/*
|
||||
final CounterGroup counterGroup
|
||||
= new CounterGroup("FileInputFormatCounters",
|
||||
"FileInputFormatCounters");
|
||||
counterGroup.addCounter(new Counter("BYTES_READ",
|
||||
"BYTES_READ",
|
||||
0));
|
||||
counters.addGroup(counterGroup);
|
||||
*/
|
||||
}
|
||||
|
||||
@Override
|
||||
public Counter getCounter(Enum<?> name) {
|
||||
return counters.findCounter(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Counter getCounter(String group, String name) {
|
||||
return counters.findCounter(group, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void progress() {}
|
||||
|
||||
@Override
|
||||
public float getProgress() {
|
||||
return 0F;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setStatus(String status) {}
|
||||
}
|
||||
|
||||
|
||||
public static class StubInMemoryWriter extends RecordWriter<Text, Text> {
|
||||
|
||||
List<Text> keys = new ArrayList<Text>();
|
||||
|
||||
List<Text> values = new ArrayList<Text>();
|
||||
|
||||
@Override
|
||||
public void write(Text key, Text value) throws IOException, InterruptedException {
|
||||
keys.add(key);
|
||||
values.add(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(TaskAttemptContext context) throws IOException, InterruptedException {
|
||||
}
|
||||
|
||||
public List<Text> keys() {
|
||||
return keys;
|
||||
}
|
||||
|
||||
public List<Text> values() {
|
||||
return values;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static TaskAttemptID getTaskAttemptID(int taskId) {
|
||||
return new TaskAttemptID("", 0, TaskType.MAP, taskId, 0);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,252 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.tools.util.TestDistCpUtils;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.junit.Test;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.AfterClass;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
public class TestCopyListing extends SimpleCopyListing {
|
||||
private static final Log LOG = LogFactory.getLog(TestCopyListing.class);
|
||||
|
||||
private static final Credentials CREDENTIALS = new Credentials();
|
||||
|
||||
private static final Configuration config = new Configuration();
|
||||
private static MiniDFSCluster cluster;
|
||||
|
||||
@BeforeClass
|
||||
public static void create() throws IOException {
|
||||
cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).format(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void destroy() {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
public TestCopyListing() {
|
||||
super(config, CREDENTIALS);
|
||||
}
|
||||
|
||||
protected TestCopyListing(Configuration configuration) {
|
||||
super(configuration, CREDENTIALS);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected long getBytesToCopy() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected long getNumberOfPaths() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleSrcToFile() {
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
fs = FileSystem.get(getConf());
|
||||
List<Path> srcPaths = new ArrayList<Path>();
|
||||
srcPaths.add(new Path("/tmp/in/1"));
|
||||
srcPaths.add(new Path("/tmp/in/2"));
|
||||
Path target = new Path("/tmp/out/1");
|
||||
TestDistCpUtils.createFile(fs, "/tmp/in/1");
|
||||
TestDistCpUtils.createFile(fs, "/tmp/in/2");
|
||||
fs.mkdirs(target);
|
||||
DistCpOptions options = new DistCpOptions(srcPaths, target);
|
||||
validatePaths(options);
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
//No errors
|
||||
|
||||
target = new Path("/tmp/out/1");
|
||||
fs.create(target).close();
|
||||
options = new DistCpOptions(srcPaths, target);
|
||||
try {
|
||||
validatePaths(options);
|
||||
Assert.fail("Invalid inputs accepted");
|
||||
} catch (InvalidInputException ignore) { }
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
|
||||
srcPaths.clear();
|
||||
srcPaths.add(new Path("/tmp/in/1"));
|
||||
fs.mkdirs(new Path("/tmp/in/1"));
|
||||
target = new Path("/tmp/out/1");
|
||||
fs.create(target).close();
|
||||
options = new DistCpOptions(srcPaths, target);
|
||||
try {
|
||||
validatePaths(options);
|
||||
Assert.fail("Invalid inputs accepted");
|
||||
} catch (InvalidInputException ignore) { }
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test input validation failed");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDuplicates() {
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
fs = FileSystem.get(getConf());
|
||||
List<Path> srcPaths = new ArrayList<Path>();
|
||||
srcPaths.add(new Path("/tmp/in/*/*"));
|
||||
TestDistCpUtils.createFile(fs, "/tmp/in/1.txt");
|
||||
TestDistCpUtils.createFile(fs, "/tmp/in/src/1.txt");
|
||||
Path target = new Path("/tmp/out");
|
||||
Path listingFile = new Path("/tmp/list");
|
||||
DistCpOptions options = new DistCpOptions(srcPaths, target);
|
||||
CopyListing listing = CopyListing.getCopyListing(getConf(), CREDENTIALS, options);
|
||||
try {
|
||||
listing.buildListing(listingFile, options);
|
||||
Assert.fail("Duplicates not detected");
|
||||
} catch (DuplicateFileException ignore) {
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered in test", e);
|
||||
Assert.fail("Test failed " + e.getMessage());
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuildListing() {
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
fs = FileSystem.get(getConf());
|
||||
List<Path> srcPaths = new ArrayList<Path>();
|
||||
Path p1 = new Path("/tmp/in/1");
|
||||
Path p2 = new Path("/tmp/in/2");
|
||||
Path p3 = new Path("/tmp/in2/2");
|
||||
Path target = new Path("/tmp/out/1");
|
||||
srcPaths.add(p1.getParent());
|
||||
srcPaths.add(p3.getParent());
|
||||
TestDistCpUtils.createFile(fs, "/tmp/in/1");
|
||||
TestDistCpUtils.createFile(fs, "/tmp/in/2");
|
||||
TestDistCpUtils.createFile(fs, "/tmp/in2/2");
|
||||
fs.mkdirs(target);
|
||||
OutputStream out = fs.create(p1);
|
||||
out.write("ABC".getBytes());
|
||||
out.close();
|
||||
|
||||
out = fs.create(p2);
|
||||
out.write("DEF".getBytes());
|
||||
out.close();
|
||||
|
||||
out = fs.create(p3);
|
||||
out.write("GHIJ".getBytes());
|
||||
out.close();
|
||||
|
||||
Path listingFile = new Path("/tmp/file");
|
||||
|
||||
DistCpOptions options = new DistCpOptions(srcPaths, target);
|
||||
options.setSyncFolder(true);
|
||||
CopyListing listing = new SimpleCopyListing(getConf(), CREDENTIALS);
|
||||
try {
|
||||
listing.buildListing(listingFile, options);
|
||||
Assert.fail("Duplicates not detected");
|
||||
} catch (DuplicateFileException ignore) {
|
||||
}
|
||||
Assert.assertEquals(listing.getBytesToCopy(), 10);
|
||||
Assert.assertEquals(listing.getNumberOfPaths(), 3);
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
|
||||
try {
|
||||
listing.buildListing(listingFile, options);
|
||||
Assert.fail("Invalid input not detected");
|
||||
} catch (InvalidInputException ignore) {
|
||||
}
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test build listing failed");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuildListingForSingleFile() {
|
||||
FileSystem fs = null;
|
||||
String testRootString = "/singleFileListing";
|
||||
Path testRoot = new Path(testRootString);
|
||||
SequenceFile.Reader reader = null;
|
||||
try {
|
||||
fs = FileSystem.get(getConf());
|
||||
if (fs.exists(testRoot))
|
||||
TestDistCpUtils.delete(fs, testRootString);
|
||||
|
||||
Path sourceFile = new Path(testRoot, "/source/foo/bar/source.txt");
|
||||
Path decoyFile = new Path(testRoot, "/target/moo/source.txt");
|
||||
Path targetFile = new Path(testRoot, "/target/moo/target.txt");
|
||||
|
||||
TestDistCpUtils.createFile(fs, sourceFile.toString());
|
||||
TestDistCpUtils.createFile(fs, decoyFile.toString());
|
||||
TestDistCpUtils.createFile(fs, targetFile.toString());
|
||||
|
||||
List<Path> srcPaths = new ArrayList<Path>();
|
||||
srcPaths.add(sourceFile);
|
||||
|
||||
DistCpOptions options = new DistCpOptions(srcPaths, targetFile);
|
||||
CopyListing listing = new SimpleCopyListing(getConf(), CREDENTIALS);
|
||||
|
||||
final Path listFile = new Path(testRoot, "/tmp/fileList.seq");
|
||||
listing.buildListing(listFile, options);
|
||||
|
||||
reader = new SequenceFile.Reader(fs, listFile, getConf());
|
||||
FileStatus fileStatus = new FileStatus();
|
||||
Text relativePath = new Text();
|
||||
Assert.assertTrue(reader.next(relativePath, fileStatus));
|
||||
Assert.assertTrue(relativePath.toString().equals(""));
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.fail("Unexpected exception encountered.");
|
||||
LOG.error("Unexpected exception: ", e);
|
||||
}
|
||||
finally {
|
||||
TestDistCpUtils.delete(fs, testRootString);
|
||||
IOUtils.closeStream(reader);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,275 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.mapred.MiniMRCluster;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.mapreduce.JobSubmissionFiles;
|
||||
import org.apache.hadoop.mapreduce.Cluster;
|
||||
import org.apache.hadoop.tools.mapred.CopyOutputFormat;
|
||||
import org.junit.*;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.io.*;
|
||||
|
||||
@Ignore
|
||||
public class TestDistCp {
|
||||
private static final Log LOG = LogFactory.getLog(TestDistCp.class);
|
||||
private static List<Path> pathList = new ArrayList<Path>();
|
||||
private static final int FILE_SIZE = 1024;
|
||||
|
||||
private static Configuration configuration;
|
||||
private static MiniDFSCluster cluster;
|
||||
private static MiniMRCluster mrCluster;
|
||||
|
||||
private static final String SOURCE_PATH = "/tmp/source";
|
||||
private static final String TARGET_PATH = "/tmp/target";
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws Exception {
|
||||
configuration = getConfigurationForCluster();
|
||||
cluster = new MiniDFSCluster.Builder(configuration).numDataNodes(1)
|
||||
.format(true).build();
|
||||
System.setProperty("org.apache.hadoop.mapred.TaskTracker", "target/tmp");
|
||||
configuration.set("org.apache.hadoop.mapred.TaskTracker", "target/tmp");
|
||||
System.setProperty("hadoop.log.dir", "target/tmp");
|
||||
configuration.set("hadoop.log.dir", "target/tmp");
|
||||
mrCluster = new MiniMRCluster(1, cluster.getFileSystem().getUri().toString(), 1);
|
||||
Configuration mrConf = mrCluster.createJobConf();
|
||||
final String mrJobTracker = mrConf.get("mapred.job.tracker");
|
||||
configuration.set("mapred.job.tracker", mrJobTracker);
|
||||
final String mrJobTrackerAddress
|
||||
= mrConf.get("mapred.job.tracker.http.address");
|
||||
configuration.set("mapred.job.tracker.http.address", mrJobTrackerAddress);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanup() {
|
||||
if (mrCluster != null) mrCluster.shutdown();
|
||||
if (cluster != null) cluster.shutdown();
|
||||
}
|
||||
|
||||
private static Configuration getConfigurationForCluster() throws IOException {
|
||||
Configuration configuration = new Configuration();
|
||||
System.setProperty("test.build.data", "target/build/TEST_DISTCP/data");
|
||||
configuration.set("hadoop.log.dir", "target/tmp");
|
||||
|
||||
LOG.debug("fs.default.name == " + configuration.get("fs.default.name"));
|
||||
LOG.debug("dfs.http.address == " + configuration.get("dfs.http.address"));
|
||||
return configuration;
|
||||
}
|
||||
|
||||
private static void createSourceData() throws Exception {
|
||||
mkdirs(SOURCE_PATH + "/1");
|
||||
mkdirs(SOURCE_PATH + "/2");
|
||||
mkdirs(SOURCE_PATH + "/2/3/4");
|
||||
mkdirs(SOURCE_PATH + "/2/3");
|
||||
mkdirs(SOURCE_PATH + "/5");
|
||||
touchFile(SOURCE_PATH + "/5/6");
|
||||
mkdirs(SOURCE_PATH + "/7");
|
||||
mkdirs(SOURCE_PATH + "/7/8");
|
||||
touchFile(SOURCE_PATH + "/7/8/9");
|
||||
}
|
||||
|
||||
private static void mkdirs(String path) throws Exception {
|
||||
FileSystem fileSystem = cluster.getFileSystem();
|
||||
final Path qualifiedPath = new Path(path).makeQualified(fileSystem.getUri(),
|
||||
fileSystem.getWorkingDirectory());
|
||||
pathList.add(qualifiedPath);
|
||||
fileSystem.mkdirs(qualifiedPath);
|
||||
}
|
||||
|
||||
private static void touchFile(String path) throws Exception {
|
||||
FileSystem fs;
|
||||
DataOutputStream outputStream = null;
|
||||
try {
|
||||
fs = cluster.getFileSystem();
|
||||
final Path qualifiedPath = new Path(path).makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory());
|
||||
final long blockSize = fs.getDefaultBlockSize() * 2;
|
||||
outputStream = fs.create(qualifiedPath, true, 0,
|
||||
(short)(fs.getDefaultReplication()*2),
|
||||
blockSize);
|
||||
outputStream.write(new byte[FILE_SIZE]);
|
||||
pathList.add(qualifiedPath);
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
private static void clearState() throws Exception {
|
||||
pathList.clear();
|
||||
cluster.getFileSystem().delete(new Path(TARGET_PATH), true);
|
||||
createSourceData();
|
||||
}
|
||||
|
||||
// @Test
|
||||
public void testUniformSizeDistCp() throws Exception {
|
||||
try {
|
||||
clearState();
|
||||
final FileSystem fileSystem = cluster.getFileSystem();
|
||||
Path sourcePath = new Path(SOURCE_PATH)
|
||||
.makeQualified(fileSystem.getUri(),
|
||||
fileSystem.getWorkingDirectory());
|
||||
List<Path> sources = new ArrayList<Path>();
|
||||
sources.add(sourcePath);
|
||||
|
||||
Path targetPath = new Path(TARGET_PATH)
|
||||
.makeQualified(fileSystem.getUri(), fileSystem.getWorkingDirectory());
|
||||
DistCpOptions options = new DistCpOptions(sources, targetPath);
|
||||
options.setAtomicCommit(true);
|
||||
options.setBlocking(false);
|
||||
Job job = new DistCp(configuration, options).execute();
|
||||
Path workDir = CopyOutputFormat.getWorkingDirectory(job);
|
||||
Path finalDir = CopyOutputFormat.getCommitDirectory(job);
|
||||
|
||||
while (!job.isComplete()) {
|
||||
if (cluster.getFileSystem().exists(workDir)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
job.waitForCompletion(true);
|
||||
Assert.assertFalse(cluster.getFileSystem().exists(workDir));
|
||||
Assert.assertTrue(cluster.getFileSystem().exists(finalDir));
|
||||
Assert.assertFalse(cluster.getFileSystem().exists(
|
||||
new Path(job.getConfiguration().get(DistCpConstants.CONF_LABEL_META_FOLDER))));
|
||||
verifyResults();
|
||||
}
|
||||
catch (Exception e) {
|
||||
LOG.error("Exception encountered", e);
|
||||
Assert.fail("Unexpected exception: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
// @Test
|
||||
public void testCleanup() {
|
||||
try {
|
||||
clearState();
|
||||
Path sourcePath = new Path("noscheme:///file");
|
||||
List<Path> sources = new ArrayList<Path>();
|
||||
sources.add(sourcePath);
|
||||
|
||||
final FileSystem fs = cluster.getFileSystem();
|
||||
Path targetPath = new Path(TARGET_PATH)
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
DistCpOptions options = new DistCpOptions(sources, targetPath);
|
||||
|
||||
Path stagingDir = JobSubmissionFiles.getStagingDir(
|
||||
new Cluster(configuration), configuration);
|
||||
stagingDir.getFileSystem(configuration).mkdirs(stagingDir);
|
||||
|
||||
try {
|
||||
new DistCp(configuration, options).execute();
|
||||
} catch (Throwable t) {
|
||||
Assert.assertEquals(stagingDir.getFileSystem(configuration).
|
||||
listStatus(stagingDir).length, 0);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("testCleanup failed " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRootPath() throws Exception {
|
||||
try {
|
||||
clearState();
|
||||
List<Path> sources = new ArrayList<Path>();
|
||||
final FileSystem fs = cluster.getFileSystem();
|
||||
sources.add(new Path("/a")
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory()));
|
||||
sources.add(new Path("/b")
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory()));
|
||||
touchFile("/a/a.txt");
|
||||
touchFile("/b/b.txt");
|
||||
|
||||
Path targetPath = new Path("/c")
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
DistCpOptions options = new DistCpOptions(sources, targetPath);
|
||||
new DistCp(configuration, options).execute();
|
||||
Assert.assertTrue(fs.exists(new Path("/c/a/a.txt")));
|
||||
Assert.assertTrue(fs.exists(new Path("/c/b/b.txt")));
|
||||
}
|
||||
catch (Exception e) {
|
||||
LOG.error("Exception encountered", e);
|
||||
Assert.fail("Unexpected exception: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDynamicDistCp() throws Exception {
|
||||
try {
|
||||
clearState();
|
||||
final FileSystem fs = cluster.getFileSystem();
|
||||
Path sourcePath = new Path(SOURCE_PATH)
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
List<Path> sources = new ArrayList<Path>();
|
||||
sources.add(sourcePath);
|
||||
|
||||
Path targetPath = new Path(TARGET_PATH)
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
DistCpOptions options = new DistCpOptions(sources, targetPath);
|
||||
options.setCopyStrategy("dynamic");
|
||||
|
||||
options.setAtomicCommit(true);
|
||||
options.setAtomicWorkPath(new Path("/work"));
|
||||
options.setBlocking(false);
|
||||
Job job = new DistCp(configuration, options).execute();
|
||||
Path workDir = CopyOutputFormat.getWorkingDirectory(job);
|
||||
Path finalDir = CopyOutputFormat.getCommitDirectory(job);
|
||||
|
||||
while (!job.isComplete()) {
|
||||
if (fs.exists(workDir)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
job.waitForCompletion(true);
|
||||
Assert.assertFalse(fs.exists(workDir));
|
||||
Assert.assertTrue(fs.exists(finalDir));
|
||||
|
||||
verifyResults();
|
||||
}
|
||||
catch (Exception e) {
|
||||
LOG.error("Exception encountered", e);
|
||||
Assert.fail("Unexpected exception: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyResults() throws Exception {
|
||||
for (Path path : pathList) {
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
|
||||
Path sourcePath = path.makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
Path targetPath
|
||||
= new Path(sourcePath.toString().replaceAll(SOURCE_PATH, TARGET_PATH));
|
||||
|
||||
Assert.assertTrue(fs.exists(targetPath));
|
||||
Assert.assertEquals(fs.isFile(sourcePath), fs.isFile(targetPath));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,542 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
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.fs.FileStatus;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.tools.util.TestDistCpUtils;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestFileBasedCopyListing {
|
||||
private static final Log LOG = LogFactory.getLog(TestFileBasedCopyListing.class);
|
||||
|
||||
private static final Credentials CREDENTIALS = new Credentials();
|
||||
|
||||
private static final Configuration config = new Configuration();
|
||||
private static MiniDFSCluster cluster;
|
||||
private static FileSystem fs;
|
||||
|
||||
@BeforeClass
|
||||
public static void create() throws IOException {
|
||||
cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).format(true)
|
||||
.build();
|
||||
fs = cluster.getFileSystem();
|
||||
buildExpectedValuesMap();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void destroy() {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
private static Map<String, String> map = new HashMap<String, String>();
|
||||
|
||||
private static void buildExpectedValuesMap() {
|
||||
map.put("/file1", "/tmp/singlefile1/file1");
|
||||
map.put("/file2", "/tmp/singlefile2/file2");
|
||||
map.put("/file3", "/tmp/multifile/file3");
|
||||
map.put("/file4", "/tmp/multifile/file4");
|
||||
map.put("/file5", "/tmp/multifile/file5");
|
||||
map.put("/multifile/file3", "/tmp/multifile/file3");
|
||||
map.put("/multifile/file4", "/tmp/multifile/file4");
|
||||
map.put("/multifile/file5", "/tmp/multifile/file5");
|
||||
map.put("/Ufile3", "/tmp/Umultifile/Ufile3");
|
||||
map.put("/Ufile4", "/tmp/Umultifile/Ufile4");
|
||||
map.put("/Ufile5", "/tmp/Umultifile/Ufile5");
|
||||
map.put("/dir1", "/tmp/singledir/dir1");
|
||||
map.put("/singledir/dir1", "/tmp/singledir/dir1");
|
||||
map.put("/dir2", "/tmp/singledir/dir2");
|
||||
map.put("/singledir/dir2", "/tmp/singledir/dir2");
|
||||
map.put("/Udir1", "/tmp/Usingledir/Udir1");
|
||||
map.put("/Udir2", "/tmp/Usingledir/Udir2");
|
||||
map.put("/dir2/file6", "/tmp/singledir/dir2/file6");
|
||||
map.put("/singledir/dir2/file6", "/tmp/singledir/dir2/file6");
|
||||
map.put("/file7", "/tmp/singledir1/dir3/file7");
|
||||
map.put("/file8", "/tmp/singledir1/dir3/file8");
|
||||
map.put("/file9", "/tmp/singledir1/dir3/file9");
|
||||
map.put("/dir3/file7", "/tmp/singledir1/dir3/file7");
|
||||
map.put("/dir3/file8", "/tmp/singledir1/dir3/file8");
|
||||
map.put("/dir3/file9", "/tmp/singledir1/dir3/file9");
|
||||
map.put("/Ufile7", "/tmp/Usingledir1/Udir3/Ufile7");
|
||||
map.put("/Ufile8", "/tmp/Usingledir1/Udir3/Ufile8");
|
||||
map.put("/Ufile9", "/tmp/Usingledir1/Udir3/Ufile9");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleFileMissingTarget() {
|
||||
caseSingleFileMissingTarget(false);
|
||||
caseSingleFileMissingTarget(true);
|
||||
}
|
||||
|
||||
private void caseSingleFileMissingTarget(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/singlefile1/file1");
|
||||
createFiles("/tmp/singlefile1/file1");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 0);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleFileTargetFile() {
|
||||
caseSingleFileTargetFile(false);
|
||||
caseSingleFileTargetFile(true);
|
||||
}
|
||||
|
||||
private void caseSingleFileTargetFile(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/singlefile1/file1");
|
||||
createFiles("/tmp/singlefile1/file1", target.toString());
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 0);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleFileTargetDir() {
|
||||
caseSingleFileTargetDir(false);
|
||||
caseSingleFileTargetDir(true);
|
||||
}
|
||||
|
||||
private void caseSingleFileTargetDir(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/singlefile2/file2");
|
||||
createFiles("/tmp/singlefile2/file2");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 1);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleDirTargetMissing() {
|
||||
caseSingleDirTargetMissing(false);
|
||||
caseSingleDirTargetMissing(true);
|
||||
}
|
||||
|
||||
private void caseSingleDirTargetMissing(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/singledir");
|
||||
mkdirs("/tmp/singledir/dir1");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 1);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleDirTargetPresent() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/singledir");
|
||||
mkdirs("/tmp/singledir/dir1");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target);
|
||||
|
||||
checkResult(listFile, 1);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateSingleDirTargetPresent() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/Usingledir");
|
||||
mkdirs("/tmp/Usingledir/Udir1");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target, true);
|
||||
|
||||
checkResult(listFile, 1);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiFileTargetPresent() {
|
||||
caseMultiFileTargetPresent(false);
|
||||
caseMultiFileTargetPresent(true);
|
||||
}
|
||||
|
||||
private void caseMultiFileTargetPresent(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
createFiles("/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 3);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiFileTargetMissing() {
|
||||
caseMultiFileTargetMissing(false);
|
||||
caseMultiFileTargetMissing(true);
|
||||
}
|
||||
|
||||
private void caseMultiFileTargetMissing(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
createFiles("/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 3);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiDirTargetPresent() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/multifile", "/tmp/singledir");
|
||||
createFiles("/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
mkdirs(target.toString(), "/tmp/singledir/dir1");
|
||||
|
||||
runTest(listFile, target);
|
||||
|
||||
checkResult(listFile, 4);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateMultiDirTargetPresent() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/Umultifile", "/tmp/Usingledir");
|
||||
createFiles("/tmp/Umultifile/Ufile3", "/tmp/Umultifile/Ufile4", "/tmp/Umultifile/Ufile5");
|
||||
mkdirs(target.toString(), "/tmp/Usingledir/Udir1");
|
||||
|
||||
runTest(listFile, target, true);
|
||||
|
||||
checkResult(listFile, 4);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiDirTargetMissing() {
|
||||
caseMultiDirTargetMissing(false);
|
||||
caseMultiDirTargetMissing(true);
|
||||
}
|
||||
|
||||
private void caseMultiDirTargetMissing(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/multifile", "/tmp/singledir");
|
||||
createFiles("/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
mkdirs("/tmp/singledir/dir1");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 4);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGlobTargetMissingSingleLevel() {
|
||||
caseGlobTargetMissingSingleLevel(false);
|
||||
caseGlobTargetMissingSingleLevel(true);
|
||||
}
|
||||
|
||||
private void caseGlobTargetMissingSingleLevel(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp1/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/*");
|
||||
createFiles("/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
createFiles("/tmp/singledir/dir2/file6");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 5);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
TestDistCpUtils.delete(fs, "/tmp1");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGlobTargetMissingMultiLevel() {
|
||||
caseGlobTargetMissingMultiLevel(false);
|
||||
caseGlobTargetMissingMultiLevel(true);
|
||||
}
|
||||
|
||||
private void caseGlobTargetMissingMultiLevel(boolean sync) {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp1/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/*/*");
|
||||
createFiles("/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
createFiles("/tmp/singledir1/dir3/file7", "/tmp/singledir1/dir3/file8",
|
||||
"/tmp/singledir1/dir3/file9");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(listFile, 6);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
TestDistCpUtils.delete(fs, "/tmp1");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGlobTargetDirMultiLevel() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp1/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/*/*");
|
||||
createFiles("/tmp/multifile/file3", "/tmp/multifile/file4", "/tmp/multifile/file5");
|
||||
createFiles("/tmp/singledir1/dir3/file7", "/tmp/singledir1/dir3/file8",
|
||||
"/tmp/singledir1/dir3/file9");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target);
|
||||
|
||||
checkResult(listFile, 6);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
TestDistCpUtils.delete(fs, "/tmp1");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateGlobTargetDirMultiLevel() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("/tmp1/listing");
|
||||
Path target = new Path("/tmp/target");
|
||||
|
||||
addEntries(listFile, "/tmp/*/*");
|
||||
createFiles("/tmp/Umultifile/Ufile3", "/tmp/Umultifile/Ufile4", "/tmp/Umultifile/Ufile5");
|
||||
createFiles("/tmp/Usingledir1/Udir3/Ufile7", "/tmp/Usingledir1/Udir3/Ufile8",
|
||||
"/tmp/Usingledir1/Udir3/Ufile9");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target, true);
|
||||
|
||||
checkResult(listFile, 6);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing build listing", e);
|
||||
Assert.fail("build listing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp");
|
||||
TestDistCpUtils.delete(fs, "/tmp1");
|
||||
}
|
||||
}
|
||||
|
||||
private void addEntries(Path listFile, String... entries) throws IOException {
|
||||
OutputStream out = fs.create(listFile);
|
||||
try {
|
||||
for (String entry : entries){
|
||||
out.write(entry.getBytes());
|
||||
out.write("\n".getBytes());
|
||||
}
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
|
||||
private void createFiles(String... entries) throws IOException {
|
||||
for (String entry : entries){
|
||||
OutputStream out = fs.create(new Path(entry));
|
||||
try {
|
||||
out.write(entry.getBytes());
|
||||
out.write("\n".getBytes());
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void mkdirs(String... entries) throws IOException {
|
||||
for (String entry : entries){
|
||||
fs.mkdirs(new Path(entry));
|
||||
}
|
||||
}
|
||||
|
||||
private void runTest(Path listFile, Path target) throws IOException {
|
||||
runTest(listFile, target, true);
|
||||
}
|
||||
|
||||
private void runTest(Path listFile, Path target, boolean sync) throws IOException {
|
||||
CopyListing listing = new FileBasedCopyListing(config, CREDENTIALS);
|
||||
DistCpOptions options = new DistCpOptions(listFile, target);
|
||||
options.setSyncFolder(sync);
|
||||
listing.buildListing(listFile, options);
|
||||
}
|
||||
|
||||
private void checkResult(Path listFile, int count) throws IOException {
|
||||
if (count == 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
int recCount = 0;
|
||||
SequenceFile.Reader reader = new SequenceFile.Reader(config,
|
||||
SequenceFile.Reader.file(listFile));
|
||||
try {
|
||||
Text relPath = new Text();
|
||||
FileStatus fileStatus = new FileStatus();
|
||||
while (reader.next(relPath, fileStatus)) {
|
||||
Assert.assertEquals(fileStatus.getPath().toUri().getPath(), map.get(relPath.toString()));
|
||||
recCount++;
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeStream(reader);
|
||||
}
|
||||
Assert.assertEquals(recCount, count);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,135 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.net.URI;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestGlobbedCopyListing {
|
||||
|
||||
private static MiniDFSCluster cluster;
|
||||
|
||||
private static final Credentials CREDENTIALS = new Credentials();
|
||||
|
||||
public static Map<String, String> expectedValues = new HashMap<String, String>();
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws Exception {
|
||||
cluster = new MiniDFSCluster(new Configuration(), 1, true, null);
|
||||
createSourceData();
|
||||
}
|
||||
|
||||
private static void createSourceData() throws Exception {
|
||||
mkdirs("/tmp/source/1");
|
||||
mkdirs("/tmp/source/2");
|
||||
mkdirs("/tmp/source/2/3");
|
||||
mkdirs("/tmp/source/2/3/4");
|
||||
mkdirs("/tmp/source/5");
|
||||
touchFile("/tmp/source/5/6");
|
||||
mkdirs("/tmp/source/7");
|
||||
mkdirs("/tmp/source/7/8");
|
||||
touchFile("/tmp/source/7/8/9");
|
||||
}
|
||||
|
||||
private static void mkdirs(String path) throws Exception {
|
||||
FileSystem fileSystem = null;
|
||||
try {
|
||||
fileSystem = cluster.getFileSystem();
|
||||
fileSystem.mkdirs(new Path(path));
|
||||
recordInExpectedValues(path);
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem);
|
||||
}
|
||||
}
|
||||
|
||||
private static void touchFile(String path) throws Exception {
|
||||
FileSystem fileSystem = null;
|
||||
DataOutputStream outputStream = null;
|
||||
try {
|
||||
fileSystem = cluster.getFileSystem();
|
||||
outputStream = fileSystem.create(new Path(path), true, 0);
|
||||
recordInExpectedValues(path);
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
private static void recordInExpectedValues(String path) throws Exception {
|
||||
FileSystem fileSystem = cluster.getFileSystem();
|
||||
Path sourcePath = new Path(fileSystem.getUri().toString() + path);
|
||||
expectedValues.put(sourcePath.toString(), DistCpUtils.getRelativePath(
|
||||
new Path("/tmp/source"), sourcePath));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRun() throws Exception {
|
||||
final URI uri = cluster.getFileSystem().getUri();
|
||||
final String pathString = uri.toString();
|
||||
Path fileSystemPath = new Path(pathString);
|
||||
Path source = new Path(fileSystemPath.toString() + "/tmp/source");
|
||||
Path target = new Path(fileSystemPath.toString() + "/tmp/target");
|
||||
Path listingPath = new Path(fileSystemPath.toString() + "/tmp/META/fileList.seq");
|
||||
DistCpOptions options = new DistCpOptions(Arrays.asList(source), target);
|
||||
|
||||
new GlobbedCopyListing(new Configuration(), CREDENTIALS).buildListing(listingPath, options);
|
||||
|
||||
verifyContents(listingPath);
|
||||
}
|
||||
|
||||
private void verifyContents(Path listingPath) throws Exception {
|
||||
SequenceFile.Reader reader = new SequenceFile.Reader(cluster.getFileSystem(),
|
||||
listingPath, new Configuration());
|
||||
Text key = new Text();
|
||||
FileStatus value = new FileStatus();
|
||||
Map<String, String> actualValues = new HashMap<String, String>();
|
||||
while (reader.next(key, value)) {
|
||||
actualValues.put(value.getPath().toString(), key.toString());
|
||||
}
|
||||
|
||||
Assert.assertEquals(expectedValues.size(), actualValues.size());
|
||||
for (Map.Entry<String, String> entry : actualValues.entrySet()) {
|
||||
Assert.assertEquals(entry.getValue(), expectedValues.get(entry.getKey()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,466 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
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.tools.util.TestDistCpUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
||||
public class TestIntegration {
|
||||
private static final Log LOG = LogFactory.getLog(TestIntegration.class);
|
||||
|
||||
private static FileSystem fs;
|
||||
|
||||
private static Path listFile;
|
||||
private static Path target;
|
||||
private static String root;
|
||||
|
||||
private static Configuration getConf() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.default.name", "file:///");
|
||||
conf.set("mapred.job.tracker", "local");
|
||||
return conf;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
try {
|
||||
fs = FileSystem.get(getConf());
|
||||
listFile = new Path("target/tmp/listing").makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory());
|
||||
target = new Path("target/tmp/target").makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory());
|
||||
root = new Path("target/tmp").makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory()).toString();
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleFileMissingTarget() {
|
||||
caseSingleFileMissingTarget(false);
|
||||
caseSingleFileMissingTarget(true);
|
||||
}
|
||||
|
||||
private void caseSingleFileMissingTarget(boolean sync) {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "singlefile1/file1");
|
||||
createFiles("singlefile1/file1");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(target, 1);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleFileTargetFile() {
|
||||
caseSingleFileTargetFile(false);
|
||||
caseSingleFileTargetFile(true);
|
||||
}
|
||||
|
||||
private void caseSingleFileTargetFile(boolean sync) {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "singlefile1/file1");
|
||||
createFiles("singlefile1/file1", target.toString());
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(target, 1);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleFileTargetDir() {
|
||||
caseSingleFileTargetDir(false);
|
||||
caseSingleFileTargetDir(true);
|
||||
}
|
||||
|
||||
private void caseSingleFileTargetDir(boolean sync) {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "singlefile2/file2");
|
||||
createFiles("singlefile2/file2");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(target, 1, "file2");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleDirTargetMissing() {
|
||||
caseSingleDirTargetMissing(false);
|
||||
caseSingleDirTargetMissing(true);
|
||||
}
|
||||
|
||||
private void caseSingleDirTargetMissing(boolean sync) {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "singledir");
|
||||
mkdirs(root + "/singledir/dir1");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(target, 1, "dir1");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleDirTargetPresent() {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "singledir");
|
||||
mkdirs(root + "/singledir/dir1");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target, false);
|
||||
|
||||
checkResult(target, 1, "singledir/dir1");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateSingleDirTargetPresent() {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "Usingledir");
|
||||
mkdirs(root + "/Usingledir/Udir1");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target, true);
|
||||
|
||||
checkResult(target, 1, "Udir1");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiFileTargetPresent() {
|
||||
caseMultiFileTargetPresent(false);
|
||||
caseMultiFileTargetPresent(true);
|
||||
}
|
||||
|
||||
private void caseMultiFileTargetPresent(boolean sync) {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "multifile/file3", "multifile/file4", "multifile/file5");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
mkdirs(target.toString());
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(target, 3, "file3", "file4", "file5");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiFileTargetMissing() {
|
||||
caseMultiFileTargetMissing(false);
|
||||
caseMultiFileTargetMissing(true);
|
||||
}
|
||||
|
||||
private void caseMultiFileTargetMissing(boolean sync) {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "multifile/file3", "multifile/file4", "multifile/file5");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
|
||||
runTest(listFile, target, sync);
|
||||
|
||||
checkResult(target, 3, "file3", "file4", "file5");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiDirTargetPresent() {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "multifile", "singledir");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
mkdirs(target.toString(), root + "/singledir/dir1");
|
||||
|
||||
runTest(listFile, target, false);
|
||||
|
||||
checkResult(target, 2, "multifile/file3", "multifile/file4", "multifile/file5", "singledir/dir1");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateMultiDirTargetPresent() {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "Umultifile", "Usingledir");
|
||||
createFiles("Umultifile/Ufile3", "Umultifile/Ufile4", "Umultifile/Ufile5");
|
||||
mkdirs(target.toString(), root + "/Usingledir/Udir1");
|
||||
|
||||
runTest(listFile, target, true);
|
||||
|
||||
checkResult(target, 4, "Ufile3", "Ufile4", "Ufile5", "Udir1");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiDirTargetMissing() {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "multifile", "singledir");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
mkdirs(root + "/singledir/dir1");
|
||||
|
||||
runTest(listFile, target, false);
|
||||
|
||||
checkResult(target, 2, "multifile/file3", "multifile/file4",
|
||||
"multifile/file5", "singledir/dir1");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateMultiDirTargetMissing() {
|
||||
|
||||
try {
|
||||
addEntries(listFile, "multifile", "singledir");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
mkdirs(root + "/singledir/dir1");
|
||||
|
||||
runTest(listFile, target, true);
|
||||
|
||||
checkResult(target, 4, "file3", "file4", "file5", "dir1");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGlobTargetMissingSingleLevel() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("target/tmp1/listing").makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory());
|
||||
addEntries(listFile, "*");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
createFiles("singledir/dir2/file6");
|
||||
|
||||
runTest(listFile, target, false);
|
||||
|
||||
checkResult(target, 2, "multifile/file3", "multifile/file4", "multifile/file5",
|
||||
"singledir/dir2/file6");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
TestDistCpUtils.delete(fs, "target/tmp1");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateGlobTargetMissingSingleLevel() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("target/tmp1/listing").makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory());
|
||||
addEntries(listFile, "*");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
createFiles("singledir/dir2/file6");
|
||||
|
||||
runTest(listFile, target, true);
|
||||
|
||||
checkResult(target, 4, "file3", "file4", "file5", "dir2/file6");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while running distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
TestDistCpUtils.delete(fs, "target/tmp1");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGlobTargetMissingMultiLevel() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("target/tmp1/listing").makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory());
|
||||
addEntries(listFile, "*/*");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
createFiles("singledir1/dir3/file7", "singledir1/dir3/file8",
|
||||
"singledir1/dir3/file9");
|
||||
|
||||
runTest(listFile, target, false);
|
||||
|
||||
checkResult(target, 4, "file3", "file4", "file5",
|
||||
"dir3/file7", "dir3/file8", "dir3/file9");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while running distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
TestDistCpUtils.delete(fs, "target/tmp1");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateGlobTargetMissingMultiLevel() {
|
||||
|
||||
try {
|
||||
Path listFile = new Path("target/tmp1/listing").makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory());
|
||||
addEntries(listFile, "*/*");
|
||||
createFiles("multifile/file3", "multifile/file4", "multifile/file5");
|
||||
createFiles("singledir1/dir3/file7", "singledir1/dir3/file8",
|
||||
"singledir1/dir3/file9");
|
||||
|
||||
runTest(listFile, target, true);
|
||||
|
||||
checkResult(target, 6, "file3", "file4", "file5",
|
||||
"file7", "file8", "file9");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while running distcp", e);
|
||||
Assert.fail("distcp failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, root);
|
||||
TestDistCpUtils.delete(fs, "target/tmp1");
|
||||
}
|
||||
}
|
||||
|
||||
private void addEntries(Path listFile, String... entries) throws IOException {
|
||||
OutputStream out = fs.create(listFile);
|
||||
try {
|
||||
for (String entry : entries){
|
||||
out.write((root + "/" + entry).getBytes());
|
||||
out.write("\n".getBytes());
|
||||
}
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
|
||||
private void createFiles(String... entries) throws IOException {
|
||||
for (String entry : entries){
|
||||
OutputStream out = fs.create(new Path(root + "/" + entry));
|
||||
try {
|
||||
out.write((root + "/" + entry).getBytes());
|
||||
out.write("\n".getBytes());
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void mkdirs(String... entries) throws IOException {
|
||||
for (String entry : entries){
|
||||
fs.mkdirs(new Path(entry));
|
||||
}
|
||||
}
|
||||
|
||||
private void runTest(Path listFile, Path target, boolean sync) throws IOException {
|
||||
DistCpOptions options = new DistCpOptions(listFile, target);
|
||||
options.setSyncFolder(sync);
|
||||
try {
|
||||
new DistCp(getConf(), options).execute();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private void checkResult(Path target, int count, String... relPaths) throws IOException {
|
||||
Assert.assertEquals(count, fs.listStatus(target).length);
|
||||
if (relPaths == null || relPaths.length == 0) {
|
||||
Assert.assertTrue(target.toString(), fs.exists(target));
|
||||
return;
|
||||
}
|
||||
for (String relPath : relPaths) {
|
||||
Assert.assertTrue(new Path(target, relPath).toString(), fs.exists(new Path(target, relPath)));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,497 @@
|
|||
/**
|
||||
* 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.hadoop.tools;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.tools.DistCpOptions.*;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
public class TestOptionsParser {
|
||||
|
||||
@Test
|
||||
public void testParseIgnoreFailure() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldIgnoreFailures());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-i",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldIgnoreFailures());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseOverwrite() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldOverwrite());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-overwrite",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldOverwrite());
|
||||
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-update",
|
||||
"-overwrite",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.fail("Update and overwrite aren't allowed together");
|
||||
} catch (IllegalArgumentException ignore) {
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLogPath() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertNull(options.getLogPath());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-log",
|
||||
"hdfs://localhost:8020/logs",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getLogPath(), new Path("hdfs://localhost:8020/logs"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseBlokcing() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldBlock());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-async",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldBlock());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsebandwidth() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getMapBandwidth(), DistCpConstants.DEFAULT_BANDWIDTH_MB);
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-bandwidth",
|
||||
"11",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getMapBandwidth(), 11);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseSkipCRC() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldSkipCRC());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-update",
|
||||
"-skipcrccheck",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldSyncFolder());
|
||||
Assert.assertTrue(options.shouldSkipCRC());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseAtomicCommit() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldAtomicCommit());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-atomic",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldAtomicCommit());
|
||||
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-atomic",
|
||||
"-update",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.fail("Atomic and sync folders were allowed");
|
||||
} catch (IllegalArgumentException ignore) { }
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseWorkPath() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertNull(options.getAtomicWorkPath());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-atomic",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertNull(options.getAtomicWorkPath());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-atomic",
|
||||
"-tmp",
|
||||
"hdfs://localhost:8020/work",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getAtomicWorkPath(), new Path("hdfs://localhost:8020/work"));
|
||||
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-tmp",
|
||||
"hdfs://localhost:8020/work",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.fail("work path was allowed without -atomic switch");
|
||||
} catch (IllegalArgumentException ignore) {}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseSyncFolders() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldSyncFolder());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-update",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldSyncFolder());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDeleteMissing() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldDeleteMissing());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-update",
|
||||
"-delete",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldSyncFolder());
|
||||
Assert.assertTrue(options.shouldDeleteMissing());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-overwrite",
|
||||
"-delete",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldOverwrite());
|
||||
Assert.assertTrue(options.shouldDeleteMissing());
|
||||
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-atomic",
|
||||
"-delete",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.fail("Atomic and delete folders were allowed");
|
||||
} catch (IllegalArgumentException ignore) { }
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseSSLConf() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertNull(options.getSslConfigurationFile());
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-mapredSslConf",
|
||||
"/tmp/ssl-client.xml",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getSslConfigurationFile(), "/tmp/ssl-client.xml");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseMaps() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getMaxMaps(), DistCpConstants.DEFAULT_MAPS);
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-m",
|
||||
"1",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getMaxMaps(), 1);
|
||||
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-m",
|
||||
"hello",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.fail("Non numberic map parsed");
|
||||
} catch (IllegalArgumentException ignore) { }
|
||||
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-mapredXslConf",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.fail("Non numberic map parsed");
|
||||
} catch (IllegalArgumentException ignore) { }
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSourceListing() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getSourceFileListing(),
|
||||
new Path("hdfs://localhost:8020/source/first"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSourceListingAndSourcePath() {
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.fail("Both source listing & source paths allowed");
|
||||
} catch (IllegalArgumentException ignore) {}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingSourceInfo() {
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.fail("Neither source listing not source paths present");
|
||||
} catch (IllegalArgumentException ignore) {}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingTarget() {
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-f", "hdfs://localhost:8020/source"});
|
||||
Assert.fail("Missing target allowed");
|
||||
} catch (IllegalArgumentException ignore) {}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidArgs() {
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-m", "-f", "hdfs://localhost:8020/source"});
|
||||
Assert.fail("Missing map value");
|
||||
} catch (IllegalArgumentException ignore) {}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToString() {
|
||||
DistCpOptions option = new DistCpOptions(new Path("abc"), new Path("xyz"));
|
||||
String val = "DistCpOptions{atomicCommit=false, syncFolder=false, deleteMissing=false, " +
|
||||
"ignoreFailures=false, maxMaps=20, sslConfigurationFile='null', copyStrategy='uniformsize', " +
|
||||
"sourceFileListing=abc, sourcePaths=null, targetPath=xyz}";
|
||||
Assert.assertEquals(val, option.toString());
|
||||
Assert.assertNotSame(DistCpOptionSwitch.ATOMIC_COMMIT.toString(),
|
||||
DistCpOptionSwitch.ATOMIC_COMMIT.name());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCopyStrategy() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"-strategy",
|
||||
"dynamic",
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getCopyStrategy(), "dynamic");
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getCopyStrategy(), DistCpConstants.UNIFORMSIZE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTargetPath() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertEquals(options.getTargetPath(), new Path("hdfs://localhost:8020/target/"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreserve() {
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.BLOCKSIZE));
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.REPLICATION));
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.USER));
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.GROUP));
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-p",
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.USER));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.GROUP));
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-p",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.USER));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.GROUP));
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-pbr",
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.USER));
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.GROUP));
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-pbrgup",
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.BLOCKSIZE));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.REPLICATION));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.USER));
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.GROUP));
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-p",
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
int i = 0;
|
||||
Iterator<FileAttribute> attribIterator = options.preserveAttributes();
|
||||
while (attribIterator.hasNext()) {
|
||||
attribIterator.next();
|
||||
i++;
|
||||
}
|
||||
Assert.assertEquals(i, 5);
|
||||
|
||||
try {
|
||||
OptionsParser.parse(new String[] {
|
||||
"-pabc",
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target"});
|
||||
Assert.fail("Invalid preserve attribute");
|
||||
}
|
||||
catch (IllegalArgumentException ignore) {}
|
||||
catch (NoSuchElementException ignore) {}
|
||||
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-f",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
Assert.assertFalse(options.shouldPreserve(FileAttribute.PERMISSION));
|
||||
options.preserve(FileAttribute.PERMISSION);
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
|
||||
|
||||
options.preserve(FileAttribute.PERMISSION);
|
||||
Assert.assertTrue(options.shouldPreserve(FileAttribute.PERMISSION));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOptionsSwitchAddToConf() {
|
||||
Configuration conf = new Configuration();
|
||||
Assert.assertNull(conf.get(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel()));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.ATOMIC_COMMIT);
|
||||
Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOptionsAppendToConf() {
|
||||
Configuration conf = new Configuration();
|
||||
Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false));
|
||||
Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
|
||||
DistCpOptions options = OptionsParser.parse(new String[] {
|
||||
"-atomic",
|
||||
"-i",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
options.appendToConf(conf);
|
||||
Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false));
|
||||
Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
|
||||
Assert.assertEquals(conf.getInt(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1),
|
||||
DistCpConstants.DEFAULT_BANDWIDTH_MB);
|
||||
|
||||
conf = new Configuration();
|
||||
Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
|
||||
Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false));
|
||||
Assert.assertEquals(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel()), null);
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-update",
|
||||
"-delete",
|
||||
"-pu",
|
||||
"-bandwidth",
|
||||
"11",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/"});
|
||||
options.appendToConf(conf);
|
||||
Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
|
||||
Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false));
|
||||
Assert.assertEquals(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel()), "U");
|
||||
Assert.assertEquals(conf.getInt(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 11);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,419 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
|
||||
import org.apache.hadoop.mapreduce.task.JobContextImpl;
|
||||
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
|
||||
import org.apache.hadoop.tools.CopyListing;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.tools.DistCpOptions;
|
||||
import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
||||
import org.apache.hadoop.tools.GlobbedCopyListing;
|
||||
import org.apache.hadoop.tools.util.TestDistCpUtils;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.junit.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
public class TestCopyCommitter {
|
||||
private static final Log LOG = LogFactory.getLog(TestCopyCommitter.class);
|
||||
|
||||
private static final Random rand = new Random();
|
||||
|
||||
private static final Credentials CREDENTIALS = new Credentials();
|
||||
public static final int PORT = 39737;
|
||||
|
||||
|
||||
private static Configuration config;
|
||||
private static MiniDFSCluster cluster;
|
||||
|
||||
private static Job getJobForClient() throws IOException {
|
||||
Job job = Job.getInstance(new Configuration());
|
||||
job.getConfiguration().set("mapred.job.tracker", "localhost:" + PORT);
|
||||
job.setInputFormatClass(NullInputFormat.class);
|
||||
job.setOutputFormatClass(NullOutputFormat.class);
|
||||
job.setNumReduceTasks(0);
|
||||
return job;
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void create() throws IOException {
|
||||
config = getJobForClient().getConfiguration();
|
||||
config.setLong(DistCpConstants.CONF_LABEL_TOTAL_BYTES_TO_BE_COPIED, 0);
|
||||
cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).format(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void destroy() {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void createMetaFolder() {
|
||||
config.set(DistCpConstants.CONF_LABEL_META_FOLDER, "/meta");
|
||||
Path meta = new Path("/meta");
|
||||
try {
|
||||
cluster.getFileSystem().mkdirs(meta);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while creating meta folder", e);
|
||||
Assert.fail("Unable to create meta folder");
|
||||
}
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanupMetaFolder() {
|
||||
Path meta = new Path("/meta");
|
||||
try {
|
||||
if (cluster.getFileSystem().exists(meta)) {
|
||||
cluster.getFileSystem().delete(meta, true);
|
||||
Assert.fail("Expected meta folder to be deleted");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while cleaning up folder", e);
|
||||
Assert.fail("Unable to clean up meta folder");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoCommitAction() {
|
||||
TaskAttemptContext taskAttemptContext = getTaskAttemptContext(config);
|
||||
JobContext jobContext = new JobContextImpl(taskAttemptContext.getConfiguration(),
|
||||
taskAttemptContext.getTaskAttemptID().getJobID());
|
||||
try {
|
||||
OutputCommitter committer = new CopyCommitter(null, taskAttemptContext);
|
||||
committer.commitJob(jobContext);
|
||||
Assert.assertEquals(taskAttemptContext.getStatus(), "Commit Successful");
|
||||
|
||||
//Test for idempotent commit
|
||||
committer.commitJob(jobContext);
|
||||
Assert.assertEquals(taskAttemptContext.getStatus(), "Commit Successful");
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Commit failed");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreserveStatus() {
|
||||
TaskAttemptContext taskAttemptContext = getTaskAttemptContext(config);
|
||||
JobContext jobContext = new JobContextImpl(taskAttemptContext.getConfiguration(),
|
||||
taskAttemptContext.getTaskAttemptID().getJobID());
|
||||
Configuration conf = jobContext.getConfiguration();
|
||||
|
||||
|
||||
String sourceBase;
|
||||
String targetBase;
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
OutputCommitter committer = new CopyCommitter(null, taskAttemptContext);
|
||||
fs = FileSystem.get(conf);
|
||||
FsPermission sourcePerm = new FsPermission((short) 511);
|
||||
FsPermission initialPerm = new FsPermission((short) 448);
|
||||
sourceBase = TestDistCpUtils.createTestSetup(fs, sourcePerm);
|
||||
targetBase = TestDistCpUtils.createTestSetup(fs, initialPerm);
|
||||
|
||||
DistCpOptions options = new DistCpOptions(Arrays.asList(new Path(sourceBase)),
|
||||
new Path("/out"));
|
||||
options.preserve(FileAttribute.PERMISSION);
|
||||
options.appendToConf(conf);
|
||||
|
||||
CopyListing listing = new GlobbedCopyListing(conf, CREDENTIALS);
|
||||
Path listingFile = new Path("/tmp1/" + String.valueOf(rand.nextLong()));
|
||||
listing.buildListing(listingFile, options);
|
||||
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, targetBase);
|
||||
|
||||
committer.commitJob(jobContext);
|
||||
if (!checkDirectoryPermissions(fs, targetBase, sourcePerm)) {
|
||||
Assert.fail("Permission don't match");
|
||||
}
|
||||
|
||||
//Test for idempotent commit
|
||||
committer.commitJob(jobContext);
|
||||
if (!checkDirectoryPermissions(fs, targetBase, sourcePerm)) {
|
||||
Assert.fail("Permission don't match");
|
||||
}
|
||||
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing for preserve status", e);
|
||||
Assert.fail("Preserve status failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp1");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteMissing() {
|
||||
TaskAttemptContext taskAttemptContext = getTaskAttemptContext(config);
|
||||
JobContext jobContext = new JobContextImpl(taskAttemptContext.getConfiguration(),
|
||||
taskAttemptContext.getTaskAttemptID().getJobID());
|
||||
Configuration conf = jobContext.getConfiguration();
|
||||
|
||||
String sourceBase;
|
||||
String targetBase;
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
OutputCommitter committer = new CopyCommitter(null, taskAttemptContext);
|
||||
fs = FileSystem.get(conf);
|
||||
sourceBase = TestDistCpUtils.createTestSetup(fs, FsPermission.getDefault());
|
||||
targetBase = TestDistCpUtils.createTestSetup(fs, FsPermission.getDefault());
|
||||
String targetBaseAdd = TestDistCpUtils.createTestSetup(fs, FsPermission.getDefault());
|
||||
fs.rename(new Path(targetBaseAdd), new Path(targetBase));
|
||||
|
||||
DistCpOptions options = new DistCpOptions(Arrays.asList(new Path(sourceBase)),
|
||||
new Path("/out"));
|
||||
options.setSyncFolder(true);
|
||||
options.setDeleteMissing(true);
|
||||
options.appendToConf(conf);
|
||||
|
||||
CopyListing listing = new GlobbedCopyListing(conf, CREDENTIALS);
|
||||
Path listingFile = new Path("/tmp1/" + String.valueOf(rand.nextLong()));
|
||||
listing.buildListing(listingFile, options);
|
||||
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, targetBase);
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH, targetBase);
|
||||
|
||||
committer.commitJob(jobContext);
|
||||
if (!TestDistCpUtils.checkIfFoldersAreInSync(fs, targetBase, sourceBase)) {
|
||||
Assert.fail("Source and target folders are not in sync");
|
||||
}
|
||||
if (!TestDistCpUtils.checkIfFoldersAreInSync(fs, sourceBase, targetBase)) {
|
||||
Assert.fail("Source and target folders are not in sync");
|
||||
}
|
||||
|
||||
//Test for idempotent commit
|
||||
committer.commitJob(jobContext);
|
||||
if (!TestDistCpUtils.checkIfFoldersAreInSync(fs, targetBase, sourceBase)) {
|
||||
Assert.fail("Source and target folders are not in sync");
|
||||
}
|
||||
if (!TestDistCpUtils.checkIfFoldersAreInSync(fs, sourceBase, targetBase)) {
|
||||
Assert.fail("Source and target folders are not in sync");
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
LOG.error("Exception encountered while testing for delete missing", e);
|
||||
Assert.fail("Delete missing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp1");
|
||||
conf.set(DistCpConstants.CONF_LABEL_DELETE_MISSING, "false");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteMissingFlatInterleavedFiles() {
|
||||
TaskAttemptContext taskAttemptContext = getTaskAttemptContext(config);
|
||||
JobContext jobContext = new JobContextImpl(taskAttemptContext.getConfiguration(),
|
||||
taskAttemptContext.getTaskAttemptID().getJobID());
|
||||
Configuration conf = jobContext.getConfiguration();
|
||||
|
||||
|
||||
String sourceBase;
|
||||
String targetBase;
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
OutputCommitter committer = new CopyCommitter(null, taskAttemptContext);
|
||||
fs = FileSystem.get(conf);
|
||||
sourceBase = "/tmp1/" + String.valueOf(rand.nextLong());
|
||||
targetBase = "/tmp1/" + String.valueOf(rand.nextLong());
|
||||
TestDistCpUtils.createFile(fs, sourceBase + "/1");
|
||||
TestDistCpUtils.createFile(fs, sourceBase + "/3");
|
||||
TestDistCpUtils.createFile(fs, sourceBase + "/4");
|
||||
TestDistCpUtils.createFile(fs, sourceBase + "/5");
|
||||
TestDistCpUtils.createFile(fs, sourceBase + "/7");
|
||||
TestDistCpUtils.createFile(fs, sourceBase + "/8");
|
||||
TestDistCpUtils.createFile(fs, sourceBase + "/9");
|
||||
|
||||
TestDistCpUtils.createFile(fs, targetBase + "/2");
|
||||
TestDistCpUtils.createFile(fs, targetBase + "/4");
|
||||
TestDistCpUtils.createFile(fs, targetBase + "/5");
|
||||
TestDistCpUtils.createFile(fs, targetBase + "/7");
|
||||
TestDistCpUtils.createFile(fs, targetBase + "/9");
|
||||
TestDistCpUtils.createFile(fs, targetBase + "/A");
|
||||
|
||||
DistCpOptions options = new DistCpOptions(Arrays.asList(new Path(sourceBase)),
|
||||
new Path("/out"));
|
||||
options.setSyncFolder(true);
|
||||
options.setDeleteMissing(true);
|
||||
options.appendToConf(conf);
|
||||
|
||||
CopyListing listing = new GlobbedCopyListing(conf, CREDENTIALS);
|
||||
Path listingFile = new Path("/tmp1/" + String.valueOf(rand.nextLong()));
|
||||
listing.buildListing(listingFile, options);
|
||||
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, targetBase);
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH, targetBase);
|
||||
|
||||
committer.commitJob(jobContext);
|
||||
if (!TestDistCpUtils.checkIfFoldersAreInSync(fs, targetBase, sourceBase)) {
|
||||
Assert.fail("Source and target folders are not in sync");
|
||||
}
|
||||
Assert.assertEquals(fs.listStatus(new Path(targetBase)).length, 4);
|
||||
|
||||
//Test for idempotent commit
|
||||
committer.commitJob(jobContext);
|
||||
if (!TestDistCpUtils.checkIfFoldersAreInSync(fs, targetBase, sourceBase)) {
|
||||
Assert.fail("Source and target folders are not in sync");
|
||||
}
|
||||
Assert.assertEquals(fs.listStatus(new Path(targetBase)).length, 4);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing for delete missing", e);
|
||||
Assert.fail("Delete missing failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, "/tmp1");
|
||||
conf.set(DistCpConstants.CONF_LABEL_DELETE_MISSING, "false");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAtomicCommitMissingFinal() {
|
||||
TaskAttemptContext taskAttemptContext = getTaskAttemptContext(config);
|
||||
JobContext jobContext = new JobContextImpl(taskAttemptContext.getConfiguration(),
|
||||
taskAttemptContext.getTaskAttemptID().getJobID());
|
||||
Configuration conf = jobContext.getConfiguration();
|
||||
|
||||
String workPath = "/tmp1/" + String.valueOf(rand.nextLong());
|
||||
String finalPath = "/tmp1/" + String.valueOf(rand.nextLong());
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
OutputCommitter committer = new CopyCommitter(null, taskAttemptContext);
|
||||
fs = FileSystem.get(conf);
|
||||
fs.mkdirs(new Path(workPath));
|
||||
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, workPath);
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH, finalPath);
|
||||
conf.setBoolean(DistCpConstants.CONF_LABEL_ATOMIC_COPY, true);
|
||||
|
||||
Assert.assertTrue(fs.exists(new Path(workPath)));
|
||||
Assert.assertFalse(fs.exists(new Path(finalPath)));
|
||||
committer.commitJob(jobContext);
|
||||
Assert.assertFalse(fs.exists(new Path(workPath)));
|
||||
Assert.assertTrue(fs.exists(new Path(finalPath)));
|
||||
|
||||
//Test for idempotent commit
|
||||
committer.commitJob(jobContext);
|
||||
Assert.assertFalse(fs.exists(new Path(workPath)));
|
||||
Assert.assertTrue(fs.exists(new Path(finalPath)));
|
||||
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing for preserve status", e);
|
||||
Assert.fail("Atomic commit failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, workPath);
|
||||
TestDistCpUtils.delete(fs, finalPath);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAtomicCommitExistingFinal() {
|
||||
TaskAttemptContext taskAttemptContext = getTaskAttemptContext(config);
|
||||
JobContext jobContext = new JobContextImpl(taskAttemptContext.getConfiguration(),
|
||||
taskAttemptContext.getTaskAttemptID().getJobID());
|
||||
Configuration conf = jobContext.getConfiguration();
|
||||
|
||||
|
||||
String workPath = "/tmp1/" + String.valueOf(rand.nextLong());
|
||||
String finalPath = "/tmp1/" + String.valueOf(rand.nextLong());
|
||||
FileSystem fs = null;
|
||||
try {
|
||||
OutputCommitter committer = new CopyCommitter(null, taskAttemptContext);
|
||||
fs = FileSystem.get(conf);
|
||||
fs.mkdirs(new Path(workPath));
|
||||
fs.mkdirs(new Path(finalPath));
|
||||
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, workPath);
|
||||
conf.set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH, finalPath);
|
||||
conf.setBoolean(DistCpConstants.CONF_LABEL_ATOMIC_COPY, true);
|
||||
|
||||
Assert.assertTrue(fs.exists(new Path(workPath)));
|
||||
Assert.assertTrue(fs.exists(new Path(finalPath)));
|
||||
try {
|
||||
committer.commitJob(jobContext);
|
||||
Assert.fail("Should not be able to atomic-commit to pre-existing path.");
|
||||
} catch(Exception exception) {
|
||||
Assert.assertTrue(fs.exists(new Path(workPath)));
|
||||
Assert.assertTrue(fs.exists(new Path(finalPath)));
|
||||
LOG.info("Atomic-commit Test pass.");
|
||||
}
|
||||
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing for atomic commit.", e);
|
||||
Assert.fail("Atomic commit failure");
|
||||
} finally {
|
||||
TestDistCpUtils.delete(fs, workPath);
|
||||
TestDistCpUtils.delete(fs, finalPath);
|
||||
}
|
||||
}
|
||||
|
||||
private TaskAttemptContext getTaskAttemptContext(Configuration conf) {
|
||||
return new TaskAttemptContextImpl(conf,
|
||||
new TaskAttemptID("200707121733", 1, TaskType.MAP, 1, 1));
|
||||
}
|
||||
|
||||
private boolean checkDirectoryPermissions(FileSystem fs, String targetBase,
|
||||
FsPermission sourcePerm) throws IOException {
|
||||
Path base = new Path(targetBase);
|
||||
|
||||
Stack<Path> stack = new Stack<Path>();
|
||||
stack.push(base);
|
||||
while (!stack.isEmpty()) {
|
||||
Path file = stack.pop();
|
||||
if (!fs.exists(file)) continue;
|
||||
FileStatus[] fStatus = fs.listStatus(file);
|
||||
if (fStatus == null || fStatus.length == 0) continue;
|
||||
|
||||
for (FileStatus status : fStatus) {
|
||||
if (status.isDirectory()) {
|
||||
stack.push(status.getPath());
|
||||
Assert.assertEquals(status.getPermission(), sourcePerm);
|
||||
}
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private static class NullInputFormat extends InputFormat {
|
||||
@Override
|
||||
public List getSplits(JobContext context)
|
||||
throws IOException, InterruptedException {
|
||||
return Collections.EMPTY_LIST;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader createRecordReader(InputSplit split,
|
||||
TaskAttemptContext context)
|
||||
throws IOException, InterruptedException {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,826 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.tools.DistCpOptionSwitch;
|
||||
import org.apache.hadoop.tools.DistCpOptions;
|
||||
import org.apache.hadoop.tools.StubContext;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
||||
public class TestCopyMapper {
|
||||
private static final Log LOG = LogFactory.getLog(TestCopyMapper.class);
|
||||
private static List<Path> pathList = new ArrayList<Path>();
|
||||
private static int nFiles = 0;
|
||||
private static final int FILE_SIZE = 1024;
|
||||
|
||||
private static MiniDFSCluster cluster;
|
||||
|
||||
private static final String SOURCE_PATH = "/tmp/source";
|
||||
private static final String TARGET_PATH = "/tmp/target";
|
||||
|
||||
private static Configuration configuration;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws Exception {
|
||||
configuration = getConfigurationForCluster();
|
||||
cluster = new MiniDFSCluster.Builder(configuration)
|
||||
.numDataNodes(1)
|
||||
.format(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
private static Configuration getConfigurationForCluster() throws IOException {
|
||||
Configuration configuration = new Configuration();
|
||||
System.setProperty("test.build.data", "target/tmp/build/TEST_COPY_MAPPER/data");
|
||||
configuration.set("hadoop.log.dir", "target/tmp");
|
||||
LOG.debug("fs.default.name == " + configuration.get("fs.default.name"));
|
||||
LOG.debug("dfs.http.address == " + configuration.get("dfs.http.address"));
|
||||
return configuration;
|
||||
}
|
||||
|
||||
private static Configuration getConfiguration() throws IOException {
|
||||
Configuration configuration = getConfigurationForCluster();
|
||||
final FileSystem fs = cluster.getFileSystem();
|
||||
Path workPath = new Path(TARGET_PATH)
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
configuration.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH,
|
||||
workPath.toString());
|
||||
configuration.set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH,
|
||||
workPath.toString());
|
||||
configuration.setBoolean(DistCpOptionSwitch.OVERWRITE.getConfigLabel(),
|
||||
false);
|
||||
configuration.setBoolean(DistCpOptionSwitch.SKIP_CRC.getConfigLabel(),
|
||||
true);
|
||||
configuration.setBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(),
|
||||
true);
|
||||
configuration.set(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel(),
|
||||
"br");
|
||||
return configuration;
|
||||
}
|
||||
|
||||
private static void createSourceData() throws Exception {
|
||||
mkdirs(SOURCE_PATH + "/1");
|
||||
mkdirs(SOURCE_PATH + "/2");
|
||||
mkdirs(SOURCE_PATH + "/2/3/4");
|
||||
mkdirs(SOURCE_PATH + "/2/3");
|
||||
mkdirs(SOURCE_PATH + "/5");
|
||||
touchFile(SOURCE_PATH + "/5/6");
|
||||
mkdirs(SOURCE_PATH + "/7");
|
||||
mkdirs(SOURCE_PATH + "/7/8");
|
||||
touchFile(SOURCE_PATH + "/7/8/9");
|
||||
}
|
||||
|
||||
private static void mkdirs(String path) throws Exception {
|
||||
FileSystem fileSystem = cluster.getFileSystem();
|
||||
final Path qualifiedPath = new Path(path).makeQualified(fileSystem.getUri(),
|
||||
fileSystem.getWorkingDirectory());
|
||||
pathList.add(qualifiedPath);
|
||||
fileSystem.mkdirs(qualifiedPath);
|
||||
}
|
||||
|
||||
private static void touchFile(String path) throws Exception {
|
||||
FileSystem fs;
|
||||
DataOutputStream outputStream = null;
|
||||
try {
|
||||
fs = cluster.getFileSystem();
|
||||
final Path qualifiedPath = new Path(path).makeQualified(fs.getUri(),
|
||||
fs.getWorkingDirectory());
|
||||
final long blockSize = fs.getDefaultBlockSize() * 2;
|
||||
outputStream = fs.create(qualifiedPath, true, 0,
|
||||
(short)(fs.getDefaultReplication()*2),
|
||||
blockSize);
|
||||
outputStream.write(new byte[FILE_SIZE]);
|
||||
pathList.add(qualifiedPath);
|
||||
++nFiles;
|
||||
|
||||
FileStatus fileStatus = fs.getFileStatus(qualifiedPath);
|
||||
System.out.println(fileStatus.getBlockSize());
|
||||
System.out.println(fileStatus.getReplication());
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRun() {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
copyMapper.setup(context);
|
||||
|
||||
for (Path path: pathList) {
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
||||
fs.getFileStatus(path), context);
|
||||
}
|
||||
|
||||
// Check that the maps worked.
|
||||
for (Path path : pathList) {
|
||||
final Path targetPath = new Path(path.toString()
|
||||
.replaceAll(SOURCE_PATH, TARGET_PATH));
|
||||
Assert.assertTrue(fs.exists(targetPath));
|
||||
Assert.assertTrue(fs.isFile(targetPath) == fs.isFile(path));
|
||||
Assert.assertEquals(fs.getFileStatus(path).getReplication(),
|
||||
fs.getFileStatus(targetPath).getReplication());
|
||||
Assert.assertEquals(fs.getFileStatus(path).getBlockSize(),
|
||||
fs.getFileStatus(targetPath).getBlockSize());
|
||||
Assert.assertTrue(!fs.isFile(targetPath) ||
|
||||
fs.getFileChecksum(targetPath).equals(
|
||||
fs.getFileChecksum(path)));
|
||||
}
|
||||
|
||||
Assert.assertEquals(pathList.size(),
|
||||
stubContext.getReporter().getCounter(CopyMapper.Counter.COPY).getValue());
|
||||
Assert.assertEquals(nFiles * FILE_SIZE,
|
||||
stubContext.getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED).getValue());
|
||||
|
||||
testCopyingExistingFiles(fs, copyMapper, context);
|
||||
for (Text value : stubContext.getWriter().values()) {
|
||||
Assert.assertTrue(value.toString() + " is not skipped", value.toString().startsWith("SKIP:"));
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
LOG.error("Unexpected exception: ", e);
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
}
|
||||
|
||||
private void testCopyingExistingFiles(FileSystem fs, CopyMapper copyMapper,
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context) {
|
||||
|
||||
try {
|
||||
for (Path path : pathList) {
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
||||
fs.getFileStatus(path), context);
|
||||
}
|
||||
|
||||
Assert.assertEquals(nFiles,
|
||||
context.getCounter(CopyMapper.Counter.SKIP).getValue());
|
||||
}
|
||||
catch (Exception exception) {
|
||||
Assert.assertTrue("Caught unexpected exception:" + exception.getMessage(),
|
||||
false);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeDirFailure() {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
Configuration configuration = context.getConfiguration();
|
||||
String workPath = new Path("hftp://localhost:1234/*/*/*/?/")
|
||||
.makeQualified(fs.getUri(), fs.getWorkingDirectory()).toString();
|
||||
configuration.set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH,
|
||||
workPath);
|
||||
copyMapper.setup(context);
|
||||
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), pathList.get(0))),
|
||||
fs.getFileStatus(pathList.get(0)), context);
|
||||
|
||||
Assert.assertTrue("There should have been an exception.", false);
|
||||
}
|
||||
catch (Exception ignore) {
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIgnoreFailures() {
|
||||
doTestIgnoreFailures(true);
|
||||
doTestIgnoreFailures(false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDirToFile() {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
mkdirs(SOURCE_PATH + "/src/file");
|
||||
touchFile(TARGET_PATH + "/src/file");
|
||||
try {
|
||||
copyMapper.setup(context);
|
||||
copyMapper.map(new Text("/src/file"),
|
||||
fs.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
||||
context);
|
||||
} catch (IOException e) {
|
||||
Assert.assertTrue(e.getMessage().startsWith("Can't replace"));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreserve() {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
UserGroupInformation tmpUser = UserGroupInformation.createRemoteUser("guest");
|
||||
|
||||
final CopyMapper copyMapper = new CopyMapper();
|
||||
|
||||
final Mapper<Text, FileStatus, Text, Text>.Context context = tmpUser.
|
||||
doAs(new PrivilegedAction<Mapper<Text, FileStatus, Text, Text>.Context>() {
|
||||
@Override
|
||||
public Mapper<Text, FileStatus, Text, Text>.Context run() {
|
||||
try {
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
return stubContext.getContext();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
||||
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
||||
|
||||
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
||||
DistCpUtils.packAttributes(preserveStatus));
|
||||
|
||||
touchFile(SOURCE_PATH + "/src/file");
|
||||
mkdirs(TARGET_PATH);
|
||||
cluster.getFileSystem().setPermission(new Path(TARGET_PATH), new FsPermission((short)511));
|
||||
|
||||
final FileSystem tmpFS = tmpUser.doAs(new PrivilegedAction<FileSystem>() {
|
||||
@Override
|
||||
public FileSystem run() {
|
||||
try {
|
||||
return FileSystem.get(configuration);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
throw new RuntimeException("Test ought to fail here");
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
tmpUser.doAs(new PrivilegedAction<Integer>() {
|
||||
@Override
|
||||
public Integer run() {
|
||||
try {
|
||||
copyMapper.setup(context);
|
||||
copyMapper.map(new Text("/src/file"),
|
||||
tmpFS.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
||||
context);
|
||||
Assert.fail("Expected copy to fail");
|
||||
} catch (AccessControlException e) {
|
||||
Assert.assertTrue("Got exception: " + e.getMessage(), true);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCopyReadableFiles() {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
UserGroupInformation tmpUser = UserGroupInformation.createRemoteUser("guest");
|
||||
|
||||
final CopyMapper copyMapper = new CopyMapper();
|
||||
|
||||
final Mapper<Text, FileStatus, Text, Text>.Context context = tmpUser.
|
||||
doAs(new PrivilegedAction<Mapper<Text, FileStatus, Text, Text>.Context>() {
|
||||
@Override
|
||||
public Mapper<Text, FileStatus, Text, Text>.Context run() {
|
||||
try {
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
return stubContext.getContext();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
touchFile(SOURCE_PATH + "/src/file");
|
||||
mkdirs(TARGET_PATH);
|
||||
cluster.getFileSystem().setPermission(new Path(SOURCE_PATH + "/src/file"),
|
||||
new FsPermission(FsAction.READ, FsAction.READ, FsAction.READ));
|
||||
cluster.getFileSystem().setPermission(new Path(TARGET_PATH), new FsPermission((short)511));
|
||||
|
||||
final FileSystem tmpFS = tmpUser.doAs(new PrivilegedAction<FileSystem>() {
|
||||
@Override
|
||||
public FileSystem run() {
|
||||
try {
|
||||
return FileSystem.get(configuration);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
throw new RuntimeException("Test ought to fail here");
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
tmpUser.doAs(new PrivilegedAction<Integer>() {
|
||||
@Override
|
||||
public Integer run() {
|
||||
try {
|
||||
copyMapper.setup(context);
|
||||
copyMapper.map(new Text("/src/file"),
|
||||
tmpFS.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
||||
context);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSkipCopyNoPerms() {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
UserGroupInformation tmpUser = UserGroupInformation.createRemoteUser("guest");
|
||||
|
||||
final CopyMapper copyMapper = new CopyMapper();
|
||||
|
||||
final StubContext stubContext = tmpUser.
|
||||
doAs(new PrivilegedAction<StubContext>() {
|
||||
@Override
|
||||
public StubContext run() {
|
||||
try {
|
||||
return new StubContext(getConfiguration(), null, 0);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
final Mapper<Text, FileStatus, Text, Text>.Context context = stubContext.getContext();
|
||||
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
||||
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
||||
|
||||
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
||||
DistCpUtils.packAttributes(preserveStatus));
|
||||
|
||||
touchFile(SOURCE_PATH + "/src/file");
|
||||
touchFile(TARGET_PATH + "/src/file");
|
||||
cluster.getFileSystem().setPermission(new Path(SOURCE_PATH + "/src/file"),
|
||||
new FsPermission(FsAction.READ, FsAction.READ, FsAction.READ));
|
||||
cluster.getFileSystem().setPermission(new Path(TARGET_PATH + "/src/file"),
|
||||
new FsPermission(FsAction.READ, FsAction.READ, FsAction.READ));
|
||||
|
||||
final FileSystem tmpFS = tmpUser.doAs(new PrivilegedAction<FileSystem>() {
|
||||
@Override
|
||||
public FileSystem run() {
|
||||
try {
|
||||
return FileSystem.get(configuration);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
throw new RuntimeException("Test ought to fail here");
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
tmpUser.doAs(new PrivilegedAction<Integer>() {
|
||||
@Override
|
||||
public Integer run() {
|
||||
try {
|
||||
copyMapper.setup(context);
|
||||
copyMapper.map(new Text("/src/file"),
|
||||
tmpFS.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
||||
context);
|
||||
Assert.assertEquals(stubContext.getWriter().values().size(), 1);
|
||||
Assert.assertTrue(stubContext.getWriter().values().get(0).toString().startsWith("SKIP"));
|
||||
Assert.assertTrue(stubContext.getWriter().values().get(0).toString().
|
||||
contains(SOURCE_PATH + "/src/file"));
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailCopyWithAccessControlException() {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
UserGroupInformation tmpUser = UserGroupInformation.createRemoteUser("guest");
|
||||
|
||||
final CopyMapper copyMapper = new CopyMapper();
|
||||
|
||||
final StubContext stubContext = tmpUser.
|
||||
doAs(new PrivilegedAction<StubContext>() {
|
||||
@Override
|
||||
public StubContext run() {
|
||||
try {
|
||||
return new StubContext(getConfiguration(), null, 0);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
||||
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
||||
|
||||
final Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
||||
DistCpUtils.packAttributes(preserveStatus));
|
||||
|
||||
touchFile(SOURCE_PATH + "/src/file");
|
||||
OutputStream out = cluster.getFileSystem().create(new Path(TARGET_PATH + "/src/file"));
|
||||
out.write("hello world".getBytes());
|
||||
out.close();
|
||||
cluster.getFileSystem().setPermission(new Path(SOURCE_PATH + "/src/file"),
|
||||
new FsPermission(FsAction.READ, FsAction.READ, FsAction.READ));
|
||||
cluster.getFileSystem().setPermission(new Path(TARGET_PATH + "/src/file"),
|
||||
new FsPermission(FsAction.READ, FsAction.READ, FsAction.READ));
|
||||
|
||||
final FileSystem tmpFS = tmpUser.doAs(new PrivilegedAction<FileSystem>() {
|
||||
@Override
|
||||
public FileSystem run() {
|
||||
try {
|
||||
return FileSystem.get(configuration);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
throw new RuntimeException("Test ought to fail here");
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
tmpUser.doAs(new PrivilegedAction<Integer>() {
|
||||
@Override
|
||||
public Integer run() {
|
||||
try {
|
||||
copyMapper.setup(context);
|
||||
copyMapper.map(new Text("/src/file"),
|
||||
tmpFS.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
||||
context);
|
||||
Assert.fail("Didn't expect the file to be copied");
|
||||
} catch (AccessControlException ignore) {
|
||||
} catch (Exception e) {
|
||||
if (e.getCause() == null || !(e.getCause() instanceof AccessControlException)) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFileToDir() {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
touchFile(SOURCE_PATH + "/src/file");
|
||||
mkdirs(TARGET_PATH + "/src/file");
|
||||
try {
|
||||
copyMapper.setup(context);
|
||||
copyMapper.map(new Text("/src/file"),
|
||||
fs.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
||||
context);
|
||||
} catch (IOException e) {
|
||||
Assert.assertTrue(e.getMessage().startsWith("Can't replace"));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Test failed: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
private void doTestIgnoreFailures(boolean ignoreFailures) {
|
||||
try {
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
Configuration configuration = context.getConfiguration();
|
||||
configuration.setBoolean(
|
||||
DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(),ignoreFailures);
|
||||
configuration.setBoolean(DistCpOptionSwitch.OVERWRITE.getConfigLabel(),
|
||||
true);
|
||||
configuration.setBoolean(DistCpOptionSwitch.SKIP_CRC.getConfigLabel(),
|
||||
true);
|
||||
copyMapper.setup(context);
|
||||
|
||||
for (Path path : pathList) {
|
||||
final FileStatus fileStatus = fs.getFileStatus(path);
|
||||
if (!fileStatus.isDirectory()) {
|
||||
fs.delete(path, true);
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
||||
fileStatus, context);
|
||||
}
|
||||
}
|
||||
if (ignoreFailures) {
|
||||
for (Text value : stubContext.getWriter().values()) {
|
||||
Assert.assertTrue(value.toString() + " is not skipped", value.toString().startsWith("FAIL:"));
|
||||
}
|
||||
}
|
||||
Assert.assertTrue("There should have been an exception.", ignoreFailures);
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue("Unexpected exception: " + e.getMessage(),
|
||||
!ignoreFailures);
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
private static void deleteState() throws IOException {
|
||||
pathList.clear();
|
||||
nFiles = 0;
|
||||
cluster.getFileSystem().delete(new Path(SOURCE_PATH), true);
|
||||
cluster.getFileSystem().delete(new Path(TARGET_PATH), true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreserveBlockSizeAndReplication() {
|
||||
testPreserveBlockSizeAndReplicationImpl(true);
|
||||
testPreserveBlockSizeAndReplicationImpl(false);
|
||||
}
|
||||
|
||||
private void testPreserveBlockSizeAndReplicationImpl(boolean preserve){
|
||||
try {
|
||||
|
||||
deleteState();
|
||||
createSourceData();
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
Configuration configuration = context.getConfiguration();
|
||||
EnumSet<DistCpOptions.FileAttribute> fileAttributes
|
||||
= EnumSet.noneOf(DistCpOptions.FileAttribute.class);
|
||||
if (preserve) {
|
||||
fileAttributes.add(DistCpOptions.FileAttribute.BLOCKSIZE);
|
||||
fileAttributes.add(DistCpOptions.FileAttribute.REPLICATION);
|
||||
}
|
||||
configuration.set(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel(),
|
||||
DistCpUtils.packAttributes(fileAttributes));
|
||||
|
||||
copyMapper.setup(context);
|
||||
|
||||
for (Path path : pathList) {
|
||||
final FileStatus fileStatus = fs.getFileStatus(path);
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
||||
fileStatus, context);
|
||||
}
|
||||
|
||||
// Check that the block-size/replication aren't preserved.
|
||||
for (Path path : pathList) {
|
||||
final Path targetPath = new Path(path.toString()
|
||||
.replaceAll(SOURCE_PATH, TARGET_PATH));
|
||||
final FileStatus source = fs.getFileStatus(path);
|
||||
final FileStatus target = fs.getFileStatus(targetPath);
|
||||
if (!source.isDirectory() ) {
|
||||
Assert.assertTrue(preserve ||
|
||||
source.getBlockSize() != target.getBlockSize());
|
||||
Assert.assertTrue(preserve ||
|
||||
source.getReplication() != target.getReplication());
|
||||
Assert.assertTrue(!preserve ||
|
||||
source.getBlockSize() == target.getBlockSize());
|
||||
Assert.assertTrue(!preserve ||
|
||||
source.getReplication() == target.getReplication());
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue("Unexpected exception: " + e.getMessage(), false);
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
private static void changeUserGroup(String user, String group)
|
||||
throws IOException {
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
FsPermission changedPermission = new FsPermission(
|
||||
FsAction.ALL, FsAction.ALL, FsAction.ALL
|
||||
);
|
||||
for (Path path : pathList)
|
||||
if (fs.isFile(path)) {
|
||||
fs.setOwner(path, user, group);
|
||||
fs.setPermission(path, changedPermission);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If a single file is being copied to a location where the file (of the same
|
||||
* name) already exists, then the file shouldn't be skipped.
|
||||
*/
|
||||
@Test
|
||||
public void testSingleFileCopy() {
|
||||
try {
|
||||
deleteState();
|
||||
touchFile(SOURCE_PATH + "/1");
|
||||
Path sourceFilePath = pathList.get(0);
|
||||
Path targetFilePath = new Path(sourceFilePath.toString().replaceAll(
|
||||
SOURCE_PATH, TARGET_PATH));
|
||||
touchFile(targetFilePath.toString());
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
context.getConfiguration().set(
|
||||
DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH,
|
||||
targetFilePath.getParent().toString()); // Parent directory.
|
||||
copyMapper.setup(context);
|
||||
|
||||
final FileStatus sourceFileStatus = fs.getFileStatus(sourceFilePath);
|
||||
|
||||
long before = fs.getFileStatus(targetFilePath).getModificationTime();
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(
|
||||
new Path(SOURCE_PATH), sourceFilePath)), sourceFileStatus, context);
|
||||
long after = fs.getFileStatus(targetFilePath).getModificationTime();
|
||||
|
||||
Assert.assertTrue("File should have been skipped", before == after);
|
||||
|
||||
context.getConfiguration().set(
|
||||
DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH,
|
||||
targetFilePath.toString()); // Specify the file path.
|
||||
copyMapper.setup(context);
|
||||
|
||||
before = fs.getFileStatus(targetFilePath).getModificationTime();
|
||||
try { Thread.sleep(2); } catch (Throwable ignore) {}
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(
|
||||
new Path(SOURCE_PATH), sourceFilePath)), sourceFileStatus, context);
|
||||
after = fs.getFileStatus(targetFilePath).getModificationTime();
|
||||
|
||||
Assert.assertTrue("File should have been overwritten.", before < after);
|
||||
|
||||
} catch (Exception exception) {
|
||||
Assert.fail("Unexpected exception: " + exception.getMessage());
|
||||
exception.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreserveUserGroup() {
|
||||
testPreserveUserGroupImpl(true);
|
||||
testPreserveUserGroupImpl(false);
|
||||
}
|
||||
|
||||
private void testPreserveUserGroupImpl(boolean preserve){
|
||||
try {
|
||||
|
||||
deleteState();
|
||||
createSourceData();
|
||||
changeUserGroup("Michael", "Corleone");
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, FileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
Configuration configuration = context.getConfiguration();
|
||||
EnumSet<DistCpOptions.FileAttribute> fileAttributes
|
||||
= EnumSet.noneOf(DistCpOptions.FileAttribute.class);
|
||||
if (preserve) {
|
||||
fileAttributes.add(DistCpOptions.FileAttribute.USER);
|
||||
fileAttributes.add(DistCpOptions.FileAttribute.GROUP);
|
||||
fileAttributes.add(DistCpOptions.FileAttribute.PERMISSION);
|
||||
}
|
||||
|
||||
configuration.set(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel(),
|
||||
DistCpUtils.packAttributes(fileAttributes));
|
||||
copyMapper.setup(context);
|
||||
|
||||
for (Path path : pathList) {
|
||||
final FileStatus fileStatus = fs.getFileStatus(path);
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
||||
fileStatus, context);
|
||||
}
|
||||
|
||||
// Check that the user/group attributes are preserved
|
||||
// (only) as necessary.
|
||||
for (Path path : pathList) {
|
||||
final Path targetPath = new Path(path.toString()
|
||||
.replaceAll(SOURCE_PATH, TARGET_PATH));
|
||||
final FileStatus source = fs.getFileStatus(path);
|
||||
final FileStatus target = fs.getFileStatus(targetPath);
|
||||
if (!source.isDirectory()) {
|
||||
Assert.assertTrue(!preserve || source.getOwner().equals(target.getOwner()));
|
||||
Assert.assertTrue(!preserve || source.getGroup().equals(target.getGroup()));
|
||||
Assert.assertTrue(!preserve || source.getPermission().equals(target.getPermission()));
|
||||
Assert.assertTrue( preserve || !source.getOwner().equals(target.getOwner()));
|
||||
Assert.assertTrue( preserve || !source.getGroup().equals(target.getGroup()));
|
||||
Assert.assertTrue( preserve || !source.getPermission().equals(target.getPermission()));
|
||||
Assert.assertTrue(source.isDirectory() ||
|
||||
source.getReplication() != target.getReplication());
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue("Unexpected exception: " + e.getMessage(), false);
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,135 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
|
||||
import org.apache.hadoop.mapreduce.task.JobContextImpl;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.junit.Test;
|
||||
import org.junit.Assert;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class TestCopyOutputFormat {
|
||||
private static final Log LOG = LogFactory.getLog(TestCopyOutputFormat.class);
|
||||
|
||||
@Test
|
||||
public void testSetCommitDirectory() {
|
||||
try {
|
||||
Job job = Job.getInstance(new Configuration());
|
||||
Assert.assertEquals(null, CopyOutputFormat.getCommitDirectory(job));
|
||||
|
||||
job.getConfiguration().set(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH, "");
|
||||
Assert.assertEquals(null, CopyOutputFormat.getCommitDirectory(job));
|
||||
|
||||
Path directory = new Path("/tmp/test");
|
||||
CopyOutputFormat.setCommitDirectory(job, directory);
|
||||
Assert.assertEquals(directory, CopyOutputFormat.getCommitDirectory(job));
|
||||
Assert.assertEquals(directory.toString(), job.getConfiguration().
|
||||
get(DistCpConstants.CONF_LABEL_TARGET_FINAL_PATH));
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while running test", e);
|
||||
Assert.fail("Failed while testing for set Commit Directory");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetWorkingDirectory() {
|
||||
try {
|
||||
Job job = Job.getInstance(new Configuration());
|
||||
Assert.assertEquals(null, CopyOutputFormat.getWorkingDirectory(job));
|
||||
|
||||
job.getConfiguration().set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, "");
|
||||
Assert.assertEquals(null, CopyOutputFormat.getWorkingDirectory(job));
|
||||
|
||||
Path directory = new Path("/tmp/test");
|
||||
CopyOutputFormat.setWorkingDirectory(job, directory);
|
||||
Assert.assertEquals(directory, CopyOutputFormat.getWorkingDirectory(job));
|
||||
Assert.assertEquals(directory.toString(), job.getConfiguration().
|
||||
get(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH));
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while running test", e);
|
||||
Assert.fail("Failed while testing for set Working Directory");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetOutputCommitter() {
|
||||
try {
|
||||
TaskAttemptContext context = new TaskAttemptContextImpl(new Configuration(),
|
||||
new TaskAttemptID("200707121733", 1, TaskType.MAP, 1, 1));
|
||||
context.getConfiguration().set("mapred.output.dir", "/out");
|
||||
Assert.assertTrue(new CopyOutputFormat().getOutputCommitter(context) instanceof CopyCommitter);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Unable to get output committer");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckOutputSpecs() {
|
||||
try {
|
||||
OutputFormat outputFormat = new CopyOutputFormat();
|
||||
Job job = Job.getInstance(new Configuration());
|
||||
JobID jobID = new JobID("200707121733", 1);
|
||||
|
||||
try {
|
||||
JobContext context = new JobContextImpl(job.getConfiguration(), jobID);
|
||||
outputFormat.checkOutputSpecs(context);
|
||||
Assert.fail("No checking for invalid work/commit path");
|
||||
} catch (IllegalStateException ignore) { }
|
||||
|
||||
CopyOutputFormat.setWorkingDirectory(job, new Path("/tmp/work"));
|
||||
try {
|
||||
JobContext context = new JobContextImpl(job.getConfiguration(), jobID);
|
||||
outputFormat.checkOutputSpecs(context);
|
||||
Assert.fail("No checking for invalid commit path");
|
||||
} catch (IllegalStateException ignore) { }
|
||||
|
||||
job.getConfiguration().set(DistCpConstants.CONF_LABEL_TARGET_WORK_PATH, "");
|
||||
CopyOutputFormat.setCommitDirectory(job, new Path("/tmp/commit"));
|
||||
try {
|
||||
JobContext context = new JobContextImpl(job.getConfiguration(), jobID);
|
||||
outputFormat.checkOutputSpecs(context);
|
||||
Assert.fail("No checking for invalid work path");
|
||||
} catch (IllegalStateException ignore) { }
|
||||
|
||||
CopyOutputFormat.setWorkingDirectory(job, new Path("/tmp/work"));
|
||||
CopyOutputFormat.setCommitDirectory(job, new Path("/tmp/commit"));
|
||||
try {
|
||||
JobContext context = new JobContextImpl(job.getConfiguration(), jobID);
|
||||
outputFormat.checkOutputSpecs(context);
|
||||
} catch (IllegalStateException ignore) {
|
||||
Assert.fail("Output spec check failed.");
|
||||
}
|
||||
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered while testing checkoutput specs", e);
|
||||
Assert.fail("Checkoutput Spec failure");
|
||||
} catch (InterruptedException e) {
|
||||
LOG.error("Exception encountered while testing checkoutput specs", e);
|
||||
Assert.fail("Checkoutput Spec failure");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,254 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.task.JobContextImpl;
|
||||
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
||||
import org.apache.hadoop.tools.CopyListing;
|
||||
import org.apache.hadoop.tools.DistCpOptions;
|
||||
import org.apache.hadoop.tools.StubContext;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
|
||||
public class TestUniformSizeInputFormat {
|
||||
private static final Log LOG
|
||||
= LogFactory.getLog(TestUniformSizeInputFormat.class);
|
||||
|
||||
private static MiniDFSCluster cluster;
|
||||
private static final int N_FILES = 20;
|
||||
private static final int SIZEOF_EACH_FILE=1024;
|
||||
private static final Random random = new Random();
|
||||
private static int totalFileSize = 0;
|
||||
|
||||
private static final Credentials CREDENTIALS = new Credentials();
|
||||
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws Exception {
|
||||
cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(1)
|
||||
.format(true).build();
|
||||
totalFileSize = 0;
|
||||
|
||||
for (int i=0; i<N_FILES; ++i)
|
||||
totalFileSize += createFile("/tmp/source/" + String.valueOf(i), SIZEOF_EACH_FILE);
|
||||
}
|
||||
|
||||
private static DistCpOptions getOptions(int nMaps) throws Exception {
|
||||
Path sourcePath = new Path(cluster.getFileSystem().getUri().toString()
|
||||
+ "/tmp/source");
|
||||
Path targetPath = new Path(cluster.getFileSystem().getUri().toString()
|
||||
+ "/tmp/target");
|
||||
|
||||
List<Path> sourceList = new ArrayList<Path>();
|
||||
sourceList.add(sourcePath);
|
||||
final DistCpOptions distCpOptions = new DistCpOptions(sourceList, targetPath);
|
||||
distCpOptions.setMaxMaps(nMaps);
|
||||
return distCpOptions;
|
||||
}
|
||||
|
||||
private static int createFile(String path, int fileSize) throws Exception {
|
||||
FileSystem fileSystem = null;
|
||||
DataOutputStream outputStream = null;
|
||||
try {
|
||||
fileSystem = cluster.getFileSystem();
|
||||
outputStream = fileSystem.create(new Path(path), true, 0);
|
||||
int size = (int) Math.ceil(fileSize + (1 - random.nextFloat()) * fileSize);
|
||||
outputStream.write(new byte[size]);
|
||||
return size;
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
public void testGetSplits(int nMaps) throws Exception {
|
||||
DistCpOptions options = getOptions(nMaps);
|
||||
Configuration configuration = new Configuration();
|
||||
configuration.set("mapred.map.tasks",
|
||||
String.valueOf(options.getMaxMaps()));
|
||||
Path listFile = new Path(cluster.getFileSystem().getUri().toString()
|
||||
+ "/tmp/testGetSplits_1/fileList.seq");
|
||||
CopyListing.getCopyListing(configuration, CREDENTIALS, options).
|
||||
buildListing(listFile, options);
|
||||
|
||||
JobContext jobContext = new JobContextImpl(configuration, new JobID());
|
||||
UniformSizeInputFormat uniformSizeInputFormat = new UniformSizeInputFormat();
|
||||
List<InputSplit> splits
|
||||
= uniformSizeInputFormat.getSplits(jobContext);
|
||||
|
||||
List<InputSplit> legacySplits = legacyGetSplits(listFile, nMaps);
|
||||
|
||||
int sizePerMap = totalFileSize/nMaps;
|
||||
|
||||
checkSplits(listFile, splits);
|
||||
checkAgainstLegacy(splits, legacySplits);
|
||||
|
||||
int doubleCheckedTotalSize = 0;
|
||||
int previousSplitSize = -1;
|
||||
for (int i=0; i<splits.size(); ++i) {
|
||||
InputSplit split = splits.get(i);
|
||||
int currentSplitSize = 0;
|
||||
RecordReader<Text, FileStatus> recordReader = uniformSizeInputFormat.createRecordReader(
|
||||
split, null);
|
||||
StubContext stubContext = new StubContext(jobContext.getConfiguration(),
|
||||
recordReader, 0);
|
||||
final TaskAttemptContext taskAttemptContext
|
||||
= stubContext.getContext();
|
||||
recordReader.initialize(split, taskAttemptContext);
|
||||
while (recordReader.nextKeyValue()) {
|
||||
Path sourcePath = recordReader.getCurrentValue().getPath();
|
||||
FileSystem fs = sourcePath.getFileSystem(configuration);
|
||||
FileStatus fileStatus [] = fs.listStatus(sourcePath);
|
||||
Assert.assertEquals(fileStatus.length, 1);
|
||||
currentSplitSize += fileStatus[0].getLen();
|
||||
}
|
||||
Assert.assertTrue(
|
||||
previousSplitSize == -1
|
||||
|| Math.abs(currentSplitSize - previousSplitSize) < 0.1*sizePerMap
|
||||
|| i == splits.size()-1);
|
||||
|
||||
doubleCheckedTotalSize += currentSplitSize;
|
||||
}
|
||||
|
||||
Assert.assertEquals(totalFileSize, doubleCheckedTotalSize);
|
||||
}
|
||||
|
||||
// From
|
||||
// http://svn.apache.org/repos/asf/hadoop/mapreduce/trunk/src/tools/org/apache/hadoop/tools/DistCp.java
|
||||
private List<InputSplit> legacyGetSplits(Path listFile, int numSplits)
|
||||
throws IOException {
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
FileStatus srcst = fs.getFileStatus(listFile);
|
||||
Configuration conf = fs.getConf();
|
||||
|
||||
ArrayList<InputSplit> splits = new ArrayList<InputSplit>(numSplits);
|
||||
FileStatus value = new FileStatus();
|
||||
Text key = new Text();
|
||||
final long targetsize = totalFileSize / numSplits;
|
||||
long pos = 0L;
|
||||
long last = 0L;
|
||||
long acc = 0L;
|
||||
long cbrem = srcst.getLen();
|
||||
SequenceFile.Reader sl = null;
|
||||
|
||||
LOG.info("Average bytes per map: " + targetsize +
|
||||
", Number of maps: " + numSplits + ", total size: " + totalFileSize);
|
||||
|
||||
try {
|
||||
sl = new SequenceFile.Reader(conf, SequenceFile.Reader.file(listFile));
|
||||
for (; sl.next(key, value); last = sl.getPosition()) {
|
||||
// if adding this split would put this split past the target size,
|
||||
// cut the last split and put this next file in the next split.
|
||||
if (acc + value.getLen() > targetsize && acc != 0) {
|
||||
long splitsize = last - pos;
|
||||
FileSplit fileSplit = new FileSplit(listFile, pos, splitsize, null);
|
||||
LOG.info ("Creating split : " + fileSplit + ", bytes in split: " + splitsize);
|
||||
splits.add(fileSplit);
|
||||
cbrem -= splitsize;
|
||||
pos = last;
|
||||
acc = 0L;
|
||||
}
|
||||
acc += value.getLen();
|
||||
}
|
||||
}
|
||||
finally {
|
||||
IOUtils.closeStream(sl);
|
||||
}
|
||||
if (cbrem != 0) {
|
||||
FileSplit fileSplit = new FileSplit(listFile, pos, cbrem, null);
|
||||
LOG.info ("Creating split : " + fileSplit + ", bytes in split: " + cbrem);
|
||||
splits.add(fileSplit);
|
||||
}
|
||||
|
||||
return splits;
|
||||
}
|
||||
|
||||
private void checkSplits(Path listFile, List<InputSplit> splits) throws IOException {
|
||||
long lastEnd = 0;
|
||||
|
||||
//Verify if each split's start is matching with the previous end and
|
||||
//we are not missing anything
|
||||
for (InputSplit split : splits) {
|
||||
FileSplit fileSplit = (FileSplit) split;
|
||||
long start = fileSplit.getStart();
|
||||
Assert.assertEquals(lastEnd, start);
|
||||
lastEnd = start + fileSplit.getLength();
|
||||
}
|
||||
|
||||
//Verify there is nothing more to read from the input file
|
||||
SequenceFile.Reader reader
|
||||
= new SequenceFile.Reader(cluster.getFileSystem().getConf(),
|
||||
SequenceFile.Reader.file(listFile));
|
||||
|
||||
try {
|
||||
reader.seek(lastEnd);
|
||||
FileStatus srcFileStatus = new FileStatus();
|
||||
Text srcRelPath = new Text();
|
||||
Assert.assertFalse(reader.next(srcRelPath, srcFileStatus));
|
||||
} finally {
|
||||
IOUtils.closeStream(reader);
|
||||
}
|
||||
}
|
||||
|
||||
private void checkAgainstLegacy(List<InputSplit> splits,
|
||||
List<InputSplit> legacySplits)
|
||||
throws IOException, InterruptedException {
|
||||
|
||||
Assert.assertEquals(legacySplits.size(), splits.size());
|
||||
for (int index = 0; index < splits.size(); index++) {
|
||||
FileSplit fileSplit = (FileSplit) splits.get(index);
|
||||
FileSplit legacyFileSplit = (FileSplit) legacySplits.get(index);
|
||||
Assert.assertEquals(fileSplit.getStart(), legacyFileSplit.getStart());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSplits() throws Exception {
|
||||
testGetSplits(9);
|
||||
for (int i=1; i<N_FILES; ++i)
|
||||
testGetSplits(i);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,162 @@
|
|||
/**
|
||||
* 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.hadoop.tools.mapred.lib;
|
||||
|
||||
import junit.framework.Assert;
|
||||
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.fs.FileStatus;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.mapreduce.task.JobContextImpl;
|
||||
import org.apache.hadoop.tools.CopyListing;
|
||||
import org.apache.hadoop.tools.DistCpOptions;
|
||||
import org.apache.hadoop.tools.StubContext;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class TestDynamicInputFormat {
|
||||
private static final Log LOG = LogFactory.getLog(TestDynamicInputFormat.class);
|
||||
private static MiniDFSCluster cluster;
|
||||
private static final int N_FILES = 1000;
|
||||
private static final int NUM_SPLITS = 7;
|
||||
|
||||
private static final Credentials CREDENTIALS = new Credentials();
|
||||
|
||||
private static List<String> expectedFilePaths = new ArrayList<String>(N_FILES);
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws Exception {
|
||||
cluster = new MiniDFSCluster.Builder(getConfigurationForCluster())
|
||||
.numDataNodes(1).format(true).build();
|
||||
|
||||
for (int i=0; i<N_FILES; ++i)
|
||||
createFile("/tmp/source/" + String.valueOf(i));
|
||||
|
||||
}
|
||||
|
||||
private static Configuration getConfigurationForCluster() {
|
||||
Configuration configuration = new Configuration();
|
||||
System.setProperty("test.build.data",
|
||||
"target/tmp/build/TEST_DYNAMIC_INPUT_FORMAT/data");
|
||||
configuration.set("hadoop.log.dir", "target/tmp");
|
||||
LOG.debug("fs.default.name == " + configuration.get("fs.default.name"));
|
||||
LOG.debug("dfs.http.address == " + configuration.get("dfs.http.address"));
|
||||
return configuration;
|
||||
}
|
||||
|
||||
private static DistCpOptions getOptions() throws Exception {
|
||||
Path sourcePath = new Path(cluster.getFileSystem().getUri().toString()
|
||||
+ "/tmp/source");
|
||||
Path targetPath = new Path(cluster.getFileSystem().getUri().toString()
|
||||
+ "/tmp/target");
|
||||
|
||||
List<Path> sourceList = new ArrayList<Path>();
|
||||
sourceList.add(sourcePath);
|
||||
DistCpOptions options = new DistCpOptions(sourceList, targetPath);
|
||||
options.setMaxMaps(NUM_SPLITS);
|
||||
return options;
|
||||
}
|
||||
|
||||
private static void createFile(String path) throws Exception {
|
||||
FileSystem fileSystem = null;
|
||||
DataOutputStream outputStream = null;
|
||||
try {
|
||||
fileSystem = cluster.getFileSystem();
|
||||
outputStream = fileSystem.create(new Path(path), true, 0);
|
||||
expectedFilePaths.add(fileSystem.listStatus(
|
||||
new Path(path))[0].getPath().toString());
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSplits() throws Exception {
|
||||
DistCpOptions options = getOptions();
|
||||
Configuration configuration = new Configuration();
|
||||
configuration.set("mapred.map.tasks",
|
||||
String.valueOf(options.getMaxMaps()));
|
||||
CopyListing.getCopyListing(configuration, CREDENTIALS, options).buildListing(
|
||||
new Path(cluster.getFileSystem().getUri().toString()
|
||||
+"/tmp/testDynInputFormat/fileList.seq"), options);
|
||||
|
||||
JobContext jobContext = new JobContextImpl(configuration, new JobID());
|
||||
DynamicInputFormat<Text, FileStatus> inputFormat =
|
||||
new DynamicInputFormat<Text, FileStatus>();
|
||||
List<InputSplit> splits = inputFormat.getSplits(jobContext);
|
||||
|
||||
int nFiles = 0;
|
||||
int taskId = 0;
|
||||
|
||||
for (InputSplit split : splits) {
|
||||
RecordReader<Text, FileStatus> recordReader =
|
||||
inputFormat.createRecordReader(split, null);
|
||||
StubContext stubContext = new StubContext(jobContext.getConfiguration(),
|
||||
recordReader, taskId);
|
||||
final TaskAttemptContext taskAttemptContext
|
||||
= stubContext.getContext();
|
||||
|
||||
recordReader.initialize(splits.get(0), taskAttemptContext);
|
||||
float previousProgressValue = 0f;
|
||||
while (recordReader.nextKeyValue()) {
|
||||
FileStatus fileStatus = recordReader.getCurrentValue();
|
||||
String source = fileStatus.getPath().toString();
|
||||
System.out.println(source);
|
||||
Assert.assertTrue(expectedFilePaths.contains(source));
|
||||
final float progress = recordReader.getProgress();
|
||||
Assert.assertTrue(progress >= previousProgressValue);
|
||||
Assert.assertTrue(progress >= 0.0f);
|
||||
Assert.assertTrue(progress <= 1.0f);
|
||||
previousProgressValue = progress;
|
||||
++nFiles;
|
||||
}
|
||||
Assert.assertTrue(recordReader.getProgress() == 1.0f);
|
||||
|
||||
++taskId;
|
||||
}
|
||||
|
||||
Assert.assertEquals(expectedFilePaths.size(), nFiles);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetSplitRatio() throws Exception {
|
||||
Assert.assertEquals(1, DynamicInputFormat.getSplitRatio(1, 1000000000));
|
||||
Assert.assertEquals(2, DynamicInputFormat.getSplitRatio(11000000, 10));
|
||||
Assert.assertEquals(4, DynamicInputFormat.getSplitRatio(30, 700));
|
||||
Assert.assertEquals(2, DynamicInputFormat.getSplitRatio(30, 200));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,220 @@
|
|||
/**
|
||||
* 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.hadoop.tools.util;
|
||||
|
||||
import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.AfterClass;
|
||||
|
||||
import java.util.EnumSet;
|
||||
import java.util.Random;
|
||||
import java.util.Stack;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
||||
public class TestDistCpUtils {
|
||||
private static final Log LOG = LogFactory.getLog(TestDistCpUtils.class);
|
||||
|
||||
private static final Configuration config = new Configuration();
|
||||
private static MiniDFSCluster cluster;
|
||||
|
||||
@BeforeClass
|
||||
public static void create() throws IOException {
|
||||
cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).format(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void destroy() {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRelativePathRoot() {
|
||||
Path root = new Path("/tmp/abc");
|
||||
Path child = new Path("/tmp/abc/xyz/file");
|
||||
Assert.assertEquals(DistCpUtils.getRelativePath(root, child), "/xyz/file");
|
||||
|
||||
root = new Path("/");
|
||||
child = new Path("/a");
|
||||
Assert.assertEquals(DistCpUtils.getRelativePath(root, child), "/a");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPackAttributes() {
|
||||
EnumSet<FileAttribute> attributes = EnumSet.noneOf(FileAttribute.class);
|
||||
Assert.assertEquals(DistCpUtils.packAttributes(attributes), "");
|
||||
|
||||
attributes.add(FileAttribute.REPLICATION);
|
||||
Assert.assertEquals(DistCpUtils.packAttributes(attributes), "R");
|
||||
Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("R"));
|
||||
|
||||
attributes.add(FileAttribute.BLOCKSIZE);
|
||||
Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RB");
|
||||
Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("RB"));
|
||||
|
||||
attributes.add(FileAttribute.USER);
|
||||
Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RBU");
|
||||
Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("RBU"));
|
||||
|
||||
attributes.add(FileAttribute.GROUP);
|
||||
Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RBUG");
|
||||
Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("RBUG"));
|
||||
|
||||
attributes.add(FileAttribute.PERMISSION);
|
||||
Assert.assertEquals(DistCpUtils.packAttributes(attributes), "RBUGP");
|
||||
Assert.assertEquals(attributes, DistCpUtils.unpackAttributes("RBUGP"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreserve() {
|
||||
try {
|
||||
FileSystem fs = FileSystem.get(config);
|
||||
EnumSet<FileAttribute> attributes = EnumSet.noneOf(FileAttribute.class);
|
||||
|
||||
|
||||
Path path = new Path("/tmp/abc");
|
||||
Path src = new Path("/tmp/src");
|
||||
fs.mkdirs(path);
|
||||
fs.mkdirs(src);
|
||||
FileStatus srcStatus = fs.getFileStatus(src);
|
||||
|
||||
FsPermission noPerm = new FsPermission((short) 0);
|
||||
fs.setPermission(path, noPerm);
|
||||
fs.setOwner(path, "nobody", "nobody");
|
||||
|
||||
DistCpUtils.preserve(fs, path, srcStatus, attributes);
|
||||
FileStatus target = fs.getFileStatus(path);
|
||||
Assert.assertEquals(target.getPermission(), noPerm);
|
||||
Assert.assertEquals(target.getOwner(), "nobody");
|
||||
Assert.assertEquals(target.getGroup(), "nobody");
|
||||
|
||||
attributes.add(FileAttribute.PERMISSION);
|
||||
DistCpUtils.preserve(fs, path, srcStatus, attributes);
|
||||
target = fs.getFileStatus(path);
|
||||
Assert.assertEquals(target.getPermission(), srcStatus.getPermission());
|
||||
Assert.assertEquals(target.getOwner(), "nobody");
|
||||
Assert.assertEquals(target.getGroup(), "nobody");
|
||||
|
||||
attributes.add(FileAttribute.GROUP);
|
||||
attributes.add(FileAttribute.USER);
|
||||
DistCpUtils.preserve(fs, path, srcStatus, attributes);
|
||||
target = fs.getFileStatus(path);
|
||||
Assert.assertEquals(target.getPermission(), srcStatus.getPermission());
|
||||
Assert.assertEquals(target.getOwner(), srcStatus.getOwner());
|
||||
Assert.assertEquals(target.getGroup(), srcStatus.getGroup());
|
||||
|
||||
fs.delete(path, true);
|
||||
fs.delete(src, true);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
Assert.fail("Preserve test failure");
|
||||
}
|
||||
}
|
||||
|
||||
private static Random rand = new Random();
|
||||
|
||||
public static String createTestSetup(FileSystem fs) throws IOException {
|
||||
return createTestSetup("/tmp1", fs, FsPermission.getDefault());
|
||||
}
|
||||
|
||||
public static String createTestSetup(FileSystem fs,
|
||||
FsPermission perm) throws IOException {
|
||||
return createTestSetup("/tmp1", fs, perm);
|
||||
}
|
||||
|
||||
public static String createTestSetup(String baseDir,
|
||||
FileSystem fs,
|
||||
FsPermission perm) throws IOException {
|
||||
String base = getBase(baseDir);
|
||||
fs.mkdirs(new Path(base + "/newTest/hello/world1"));
|
||||
fs.mkdirs(new Path(base + "/newTest/hello/world2/newworld"));
|
||||
fs.mkdirs(new Path(base + "/newTest/hello/world3/oldworld"));
|
||||
fs.setPermission(new Path(base + "/newTest"), perm);
|
||||
fs.setPermission(new Path(base + "/newTest/hello"), perm);
|
||||
fs.setPermission(new Path(base + "/newTest/hello/world1"), perm);
|
||||
fs.setPermission(new Path(base + "/newTest/hello/world2"), perm);
|
||||
fs.setPermission(new Path(base + "/newTest/hello/world2/newworld"), perm);
|
||||
fs.setPermission(new Path(base + "/newTest/hello/world3"), perm);
|
||||
fs.setPermission(new Path(base + "/newTest/hello/world3/oldworld"), perm);
|
||||
createFile(fs, base + "/newTest/1");
|
||||
createFile(fs, base + "/newTest/hello/2");
|
||||
createFile(fs, base + "/newTest/hello/world3/oldworld/3");
|
||||
createFile(fs, base + "/newTest/hello/world2/4");
|
||||
return base;
|
||||
}
|
||||
|
||||
private static String getBase(String base) {
|
||||
String location = String.valueOf(rand.nextLong());
|
||||
return base + "/" + location;
|
||||
}
|
||||
|
||||
public static void delete(FileSystem fs, String path) {
|
||||
try {
|
||||
if (fs != null) {
|
||||
if (path != null) {
|
||||
fs.delete(new Path(path), true);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception encountered ", e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void createFile(FileSystem fs, String filePath) throws IOException {
|
||||
OutputStream out = fs.create(new Path(filePath));
|
||||
IOUtils.closeStream(out);
|
||||
}
|
||||
|
||||
public static boolean checkIfFoldersAreInSync(FileSystem fs, String targetBase, String sourceBase)
|
||||
throws IOException {
|
||||
Path base = new Path(targetBase);
|
||||
|
||||
Stack<Path> stack = new Stack<Path>();
|
||||
stack.push(base);
|
||||
while (!stack.isEmpty()) {
|
||||
Path file = stack.pop();
|
||||
if (!fs.exists(file)) continue;
|
||||
FileStatus[] fStatus = fs.listStatus(file);
|
||||
if (fStatus == null || fStatus.length == 0) continue;
|
||||
|
||||
for (FileStatus status : fStatus) {
|
||||
if (status.isDirectory()) {
|
||||
stack.push(status.getPath());
|
||||
}
|
||||
Assert.assertTrue(fs.exists(new Path(sourceBase + "/" +
|
||||
DistCpUtils.getRelativePath(new Path(targetBase), status.getPath()))));
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,81 @@
|
|||
/**
|
||||
* 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.hadoop.tools.util;
|
||||
|
||||
import org.apache.hadoop.io.retry.RetryPolicy;
|
||||
import org.apache.hadoop.io.retry.RetryPolicies;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class TestRetriableCommand {
|
||||
|
||||
private static class MyRetriableCommand extends RetriableCommand {
|
||||
|
||||
private int succeedAfter;
|
||||
private int retryCount = 0;
|
||||
|
||||
public MyRetriableCommand(int succeedAfter) {
|
||||
super("MyRetriableCommand");
|
||||
this.succeedAfter = succeedAfter;
|
||||
}
|
||||
|
||||
public MyRetriableCommand(int succeedAfter, RetryPolicy retryPolicy) {
|
||||
super("MyRetriableCommand", retryPolicy);
|
||||
this.succeedAfter = succeedAfter;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Object doExecute(Object... arguments) throws Exception {
|
||||
if (++retryCount < succeedAfter)
|
||||
throw new Exception("Transient failure#" + retryCount);
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRetriableCommand() {
|
||||
try {
|
||||
new MyRetriableCommand(5).execute(0);
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue(true);
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
new MyRetriableCommand(3).execute(0);
|
||||
Assert.assertTrue(true);
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
|
||||
try {
|
||||
new MyRetriableCommand(5, RetryPolicies.
|
||||
retryUpToMaximumCountWithFixedSleep(5, 0, TimeUnit.MILLISECONDS)).execute(0);
|
||||
Assert.assertTrue(true);
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,157 @@
|
|||
/**
|
||||
* 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.hadoop.tools.util;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.*;
|
||||
|
||||
public class TestThrottledInputStream {
|
||||
private static final Log LOG = LogFactory.getLog(TestThrottledInputStream.class);
|
||||
private static final int BUFF_SIZE = 1024;
|
||||
|
||||
private enum CB {ONE_C, BUFFER, BUFF_OFFSET}
|
||||
|
||||
@Test
|
||||
public void testRead() {
|
||||
File tmpFile;
|
||||
File outFile;
|
||||
try {
|
||||
tmpFile = createFile(1024);
|
||||
outFile = createFile();
|
||||
|
||||
tmpFile.deleteOnExit();
|
||||
outFile.deleteOnExit();
|
||||
|
||||
long maxBandwidth = copyAndAssert(tmpFile, outFile, 0, 1, -1, CB.BUFFER);
|
||||
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 20, 0, CB.BUFFER);
|
||||
/*
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 10, 0, CB.BUFFER);
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 50, 0, CB.BUFFER);
|
||||
*/
|
||||
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 20, 0, CB.BUFF_OFFSET);
|
||||
/*
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 10, 0, CB.BUFF_OFFSET);
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 50, 0, CB.BUFF_OFFSET);
|
||||
*/
|
||||
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 20, 0, CB.ONE_C);
|
||||
/*
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 10, 0, CB.ONE_C);
|
||||
copyAndAssert(tmpFile, outFile, maxBandwidth, 50, 0, CB.ONE_C);
|
||||
*/
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception encountered ", e);
|
||||
}
|
||||
}
|
||||
|
||||
private long copyAndAssert(File tmpFile, File outFile,
|
||||
long maxBandwidth, float factor,
|
||||
int sleepTime, CB flag) throws IOException {
|
||||
long bandwidth;
|
||||
ThrottledInputStream in;
|
||||
long maxBPS = (long) (maxBandwidth / factor);
|
||||
|
||||
if (maxBandwidth == 0) {
|
||||
in = new ThrottledInputStream(new FileInputStream(tmpFile));
|
||||
} else {
|
||||
in = new ThrottledInputStream(new FileInputStream(tmpFile), maxBPS);
|
||||
}
|
||||
OutputStream out = new FileOutputStream(outFile);
|
||||
try {
|
||||
if (flag == CB.BUFFER) {
|
||||
copyBytes(in, out, BUFF_SIZE);
|
||||
} else if (flag == CB.BUFF_OFFSET){
|
||||
copyBytesWithOffset(in, out, BUFF_SIZE);
|
||||
} else {
|
||||
copyByteByByte(in, out);
|
||||
}
|
||||
|
||||
LOG.info(in);
|
||||
bandwidth = in.getBytesPerSec();
|
||||
Assert.assertEquals(in.getTotalBytesRead(), tmpFile.length());
|
||||
Assert.assertTrue(in.getBytesPerSec() > maxBandwidth / (factor * 1.2));
|
||||
Assert.assertTrue(in.getTotalSleepTime() > sleepTime || in.getBytesPerSec() <= maxBPS);
|
||||
} finally {
|
||||
IOUtils.closeStream(in);
|
||||
IOUtils.closeStream(out);
|
||||
}
|
||||
return bandwidth;
|
||||
}
|
||||
|
||||
private static void copyBytesWithOffset(InputStream in, OutputStream out, int buffSize)
|
||||
throws IOException {
|
||||
|
||||
byte buf[] = new byte[buffSize];
|
||||
int bytesRead = in.read(buf, 0, buffSize);
|
||||
while (bytesRead >= 0) {
|
||||
out.write(buf, 0, bytesRead);
|
||||
bytesRead = in.read(buf);
|
||||
}
|
||||
}
|
||||
|
||||
private static void copyByteByByte(InputStream in, OutputStream out)
|
||||
throws IOException {
|
||||
|
||||
int ch = in.read();
|
||||
while (ch >= 0) {
|
||||
out.write(ch);
|
||||
ch = in.read();
|
||||
}
|
||||
}
|
||||
|
||||
private static void copyBytes(InputStream in, OutputStream out, int buffSize)
|
||||
throws IOException {
|
||||
|
||||
byte buf[] = new byte[buffSize];
|
||||
int bytesRead = in.read(buf);
|
||||
while (bytesRead >= 0) {
|
||||
out.write(buf, 0, bytesRead);
|
||||
bytesRead = in.read(buf);
|
||||
}
|
||||
}
|
||||
|
||||
private File createFile(long sizeInKB) throws IOException {
|
||||
File tmpFile = createFile();
|
||||
writeToFile(tmpFile, sizeInKB);
|
||||
return tmpFile;
|
||||
}
|
||||
|
||||
private File createFile() throws IOException {
|
||||
return File.createTempFile("tmp", "dat");
|
||||
}
|
||||
|
||||
private void writeToFile(File tmpFile, long sizeInKB) throws IOException {
|
||||
OutputStream out = new FileOutputStream(tmpFile);
|
||||
try {
|
||||
byte[] buffer = new byte [1024];
|
||||
for (long index = 0; index < sizeInKB; index++) {
|
||||
out.write(buffer);
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeStream(out);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,57 @@
|
|||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
|
||||
<configuration>
|
||||
|
||||
<property>
|
||||
<name>ssl.client.truststore.location</name>
|
||||
<value>/path/to/truststore/keys/keystore.jks</value>
|
||||
<description>Truststore to be used by clients like distcp. Must be
|
||||
specified.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ssl.client.truststore.password</name>
|
||||
<value>changeit</value>
|
||||
<description>Optional. Default value is "".
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ssl.client.truststore.type</name>
|
||||
<value>jks</value>
|
||||
<description>Optional. Default value is "jks".
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ssl.client.keystore.location</name>
|
||||
<value>/path/to/keystore/keys/keystore.jks</value>
|
||||
<description>Keystore to be used by clients like distcp. Must be
|
||||
specified.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ssl.client.keystore.password</name>
|
||||
<value>changeit</value>
|
||||
<description>Optional. Default value is "".
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ssl.client.keystore.keypassword</name>
|
||||
<value>changeit</value>
|
||||
<description>Optional. Default value is "".
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ssl.client.keystore.type</name>
|
||||
<value>jks</value>
|
||||
<description>Optional. Default value is "jks".
|
||||
</description>
|
||||
</property>
|
||||
|
||||
</configuration>
|
|
@ -29,6 +29,7 @@
|
|||
|
||||
<modules>
|
||||
<module>hadoop-streaming</module>
|
||||
<module>hadoop-distcp</module>
|
||||
<module>hadoop-archives</module>
|
||||
<module>hadoop-rumen</module>
|
||||
<module>hadoop-tools-dist</module>
|
||||
|
|
Loading…
Reference in New Issue