diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 69aeb9e7914..76447a77e3c 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -78,6 +78,9 @@ Release 0.23.1 - Unreleased IMPROVEMENTS + HADOOP-8015. ChRootFileSystem should extend FilterFileSystem + (Daryn Sharp via bobby) + HADOOP-7801. HADOOP_PREFIX cannot be overriden. (Bruno Mahé via tomwhite) HADOOP-7802. Hadoop scripts unconditionally source @@ -158,6 +161,14 @@ Release 0.23.1 - Unreleased OPTIMIZATIONS BUG FIXES + HADOOP-8013. ViewFileSystem does not honor setVerifyChecksum + (Daryn Sharp via bobby) + + HADOOP-8018. Hudson auto test for HDFS has started throwing javadoc + (Jon Eagles via bobby) + + HADOOP-8001 ChecksumFileSystem's rename doesn't correctly handle checksum + files. (Daryn Sharp via bobby) HADOOP-8006 TestFSInputChecker is failing in trunk. (Daryn Sharp via bobby) @@ -258,6 +269,9 @@ Release 0.23.1 - Unreleased HADOOP-7999. "hadoop archive" fails with ClassNotFoundException. (Jason Lowe via mahadev) + HADOOP-8012. hadoop-daemon.sh and yarn-daemon.sh are trying to mkdir + and chown log/pid dirs which can fail. (Roman Shaposhnik via eli) + Release 0.23.0 - 2011-11-01 INCOMPATIBLE CHANGES diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh index a843c93cbf6..abd4dfd4781 100755 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh @@ -95,8 +95,11 @@ fi if [ "$HADOOP_LOG_DIR" = "" ]; then export HADOOP_LOG_DIR="$HADOOP_PREFIX/logs" fi -mkdir -p "$HADOOP_LOG_DIR" -chown $HADOOP_IDENT_STRING $HADOOP_LOG_DIR + +if [ ! -w "$HADOOP_LOG_DIR" ] ; then + mkdir -p "$HADOOP_LOG_DIR" + chown $HADOOP_IDENT_STRING $HADOOP_LOG_DIR +fi if [ "$HADOOP_PID_DIR" = "" ]; then HADOOP_PID_DIR=/tmp @@ -118,7 +121,7 @@ case $startStop in (start) - mkdir -p "$HADOOP_PID_DIR" + [ -w "$HADOOP_PID_DIR" ] || mkdir -p "$HADOOP_PID_DIR" if [ -f $pid ]; then if kill -0 `cat $pid` > /dev/null 2>&1; then diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java index 040f59dbb8c..e122f25dcdd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java @@ -474,18 +474,21 @@ public abstract class ChecksumFileSystem extends FilterFileSystem { if (fs.isDirectory(src)) { return fs.rename(src, dst); } else { + if (fs.isDirectory(dst)) { + dst = new Path(dst, src.getName()); + } boolean value = fs.rename(src, dst); if (!value) return false; - Path checkFile = getChecksumFile(src); - if (fs.exists(checkFile)) { //try to rename checksum - if (fs.isDirectory(dst)) { - value = fs.rename(checkFile, dst); - } else { - value = fs.rename(checkFile, getChecksumFile(dst)); - } + Path srcCheckFile = getChecksumFile(src); + Path dstCheckFile = getChecksumFile(dst); + if (fs.exists(srcCheckFile)) { //try to rename checksum + value = fs.rename(srcCheckFile, dstCheckFile); + } else if (fs.exists(dstCheckFile)) { + // no src checksum, so remove dst checksum + value = fs.delete(dstCheckFile, true); } return value; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java index cedf802228c..dde2520041e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java @@ -20,6 +20,7 @@ package org.apache.hadoop.fs; import java.io.*; import java.net.URI; +import java.net.URISyntaxException; import java.util.EnumSet; import java.util.List; @@ -51,6 +52,7 @@ import org.apache.hadoop.util.Progressable; public class FilterFileSystem extends FileSystem { protected FileSystem fs; + private String swapScheme; /* * so that extending classes can define it @@ -63,13 +65,25 @@ public class FilterFileSystem extends FileSystem { this.statistics = fs.statistics; } + /** + * Get the raw file system + * @return FileSystem being filtered + */ + public FileSystem getRawFileSystem() { + return fs; + } + /** Called after a new FileSystem instance is constructed. * @param name a uri whose authority section names the host, port, etc. * for this FileSystem * @param conf the configuration */ public void initialize(URI name, Configuration conf) throws IOException { - fs.initialize(name, conf); + super.initialize(name, conf); + String scheme = name.getScheme(); + if (!scheme.equals(fs.getUri().getScheme())) { + swapScheme = scheme; + } } /** Returns a URI whose scheme and authority identify this FileSystem.*/ @@ -88,7 +102,19 @@ public class FilterFileSystem extends FileSystem { /** Make sure that a path specifies a FileSystem. */ public Path makeQualified(Path path) { - return fs.makeQualified(path); + Path fqPath = fs.makeQualified(path); + // swap in our scheme if the filtered fs is using a different scheme + if (swapScheme != null) { + try { + // NOTE: should deal with authority, but too much other stuff is broken + fqPath = new Path( + new URI(swapScheme, fqPath.toUri().getSchemeSpecificPart(), null) + ); + } catch (URISyntaxException e) { + throw new IllegalArgumentException(e); + } + } + return fqPath; } /////////////////////////////////////////////////////////////// diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java index 9e6c5d665cf..88ee7b82242 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java @@ -24,6 +24,7 @@ import java.util.*; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; /**************************************************************** * Implement the FileSystem API for the checksumed local filesystem. @@ -34,21 +35,26 @@ import org.apache.hadoop.classification.InterfaceStability; public class LocalFileSystem extends ChecksumFileSystem { static final URI NAME = URI.create("file:///"); static private Random rand = new Random(); - FileSystem rfs; public LocalFileSystem() { this(new RawLocalFileSystem()); } public FileSystem getRaw() { - return rfs; + return getRawFileSystem(); } public LocalFileSystem(FileSystem rawLocalFileSystem) { super(rawLocalFileSystem); - rfs = rawLocalFileSystem; } + @Override + public void initialize(URI uri, Configuration conf) throws IOException { + super.initialize(uri, conf); + // ctor didn't initialize the filtered fs + getRawFileSystem().initialize(uri, conf); + } + /** Convert a path to a File. */ public File pathToFile(Path path) { return ((RawLocalFileSystem)fs).pathToFile(path); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java index 580b682e430..18ec724b7aa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java @@ -19,9 +19,6 @@ package org.apache.hadoop.fs.viewfs; import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; -import java.net.URISyntaxException; -import java.util.List; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -31,11 +28,11 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.FsStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Progressable; /** @@ -60,15 +57,14 @@ import org.apache.hadoop.util.Progressable; @InterfaceAudience.Private @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */ -class ChRootedFileSystem extends FileSystem { - private final FileSystem myFs; // the base file system whose root is changed +class ChRootedFileSystem extends FilterFileSystem { private final URI myUri; // the base URI + the chRoot private final Path chRootPathPart; // the root below the root of the base private final String chRootPathPartString; private Path workingDir; protected FileSystem getMyFs() { - return myFs; + return getRawFileSystem(); } /** @@ -84,37 +80,16 @@ class ChRootedFileSystem extends FileSystem { /** * Constructor - * @param fs base file system - * @param theRoot chRoot for this file system - * @throws URISyntaxException + * @param uri base file system + * @param conf configuration + * @throws IOException */ - public ChRootedFileSystem(final FileSystem fs, final Path theRoot) - throws URISyntaxException { - myFs = fs; - myFs.makeQualified(theRoot); //check that root is a valid path for fs - // Would like to call myFs.checkPath(theRoot); - // but not public - chRootPathPart = new Path(theRoot.toUri().getPath()); + public ChRootedFileSystem(final URI uri, Configuration conf) + throws IOException { + super(FileSystem.get(uri, conf)); + chRootPathPart = new Path(uri.getPath()); chRootPathPartString = chRootPathPart.toUri().getPath(); - try { - initialize(fs.getUri(), fs.getConf()); - } catch (IOException e) { // This exception should not be thrown - throw new RuntimeException("This should not occur"); - } - - /* - * We are making URI include the chrootedPath: e.g. file:///chrootedPath. - * This is questionable since Path#makeQualified(uri, path) ignores - * the pathPart of a uri. Since this class is internal we can ignore - * this issue but if we were to make it external then this needs - * to be resolved. - */ - // Handle the two cases: - // scheme:/// and scheme://authority/ - myUri = new URI(myFs.getUri().toString() + - (myFs.getUri().getAuthority() == null ? "" : Path.SEPARATOR) + - chRootPathPart.toString().substring(1)); - + myUri = uri; workingDir = getHomeDirectory(); // We don't use the wd of the myFs } @@ -127,7 +102,6 @@ class ChRootedFileSystem extends FileSystem { */ public void initialize(final URI name, final Configuration conf) throws IOException { - myFs.initialize(name, conf); super.initialize(name, conf); setConf(conf); } @@ -137,12 +111,6 @@ class ChRootedFileSystem extends FileSystem { return myUri; } - @Override - public Path makeQualified(final Path path) { - return myFs.makeQualified(path); - // NOT myFs.makeQualified(fullPath(path)); - } - /** * Strip out the root from the path. * @param p - fully qualified path p @@ -175,7 +143,7 @@ class ChRootedFileSystem extends FileSystem { public Path getResolvedQualifiedPath(final Path f) throws FileNotFoundException { - return myFs.makeQualified( + return makeQualified( new Path(chRootPathPartString + f.toUri().toString())); } @@ -199,14 +167,14 @@ class ChRootedFileSystem extends FileSystem { public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize, final short replication, final long blockSize, final Progressable progress) throws IOException { - return myFs.create(fullPath(f), permission, overwrite, bufferSize, + return super.create(fullPath(f), permission, overwrite, bufferSize, replication, blockSize, progress); } @Override public boolean delete(final Path f, final boolean recursive) throws IOException { - return myFs.delete(fullPath(f), recursive); + return super.delete(fullPath(f), recursive); } @@ -219,95 +187,90 @@ class ChRootedFileSystem extends FileSystem { @Override public BlockLocation[] getFileBlockLocations(final FileStatus fs, final long start, final long len) throws IOException { - return myFs.getFileBlockLocations( + return super.getFileBlockLocations( new ViewFsFileStatus(fs, fullPath(fs.getPath())), start, len); } @Override public FileChecksum getFileChecksum(final Path f) throws IOException { - return myFs.getFileChecksum(fullPath(f)); + return super.getFileChecksum(fullPath(f)); } @Override public FileStatus getFileStatus(final Path f) throws IOException { - return myFs.getFileStatus(fullPath(f)); + return super.getFileStatus(fullPath(f)); } @Override public FsStatus getStatus(Path p) throws IOException { - return myFs.getStatus(fullPath(p)); + return super.getStatus(fullPath(p)); } @Override public FsServerDefaults getServerDefaults() throws IOException { - return myFs.getServerDefaults(); + return super.getServerDefaults(); } @Override public FileStatus[] listStatus(final Path f) throws IOException { - return myFs.listStatus(fullPath(f)); + return super.listStatus(fullPath(f)); } @Override public boolean mkdirs(final Path f, final FsPermission permission) throws IOException { - return myFs.mkdirs(fullPath(f), permission); + return super.mkdirs(fullPath(f), permission); } @Override public FSDataInputStream open(final Path f, final int bufferSize) throws IOException { - return myFs.open(fullPath(f), bufferSize); + return super.open(fullPath(f), bufferSize); } @Override public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { - return myFs.append(fullPath(f), bufferSize, progress); + return super.append(fullPath(f), bufferSize, progress); } @Override public boolean rename(final Path src, final Path dst) throws IOException { // note fullPath will check that paths are relative to this FileSystem. // Hence both are in same file system and a rename is valid - return myFs.rename(fullPath(src), fullPath(dst)); + return super.rename(fullPath(src), fullPath(dst)); } @Override public void setOwner(final Path f, final String username, final String groupname) throws IOException { - myFs.setOwner(fullPath(f), username, groupname); + super.setOwner(fullPath(f), username, groupname); } @Override public void setPermission(final Path f, final FsPermission permission) throws IOException { - myFs.setPermission(fullPath(f), permission); + super.setPermission(fullPath(f), permission); } @Override public boolean setReplication(final Path f, final short replication) throws IOException { - return myFs.setReplication(fullPath(f), replication); + return super.setReplication(fullPath(f), replication); } @Override public void setTimes(final Path f, final long mtime, final long atime) throws IOException { - myFs.setTimes(fullPath(f), mtime, atime); - } - - @Override - public void setVerifyChecksum(final boolean verifyChecksum) { - myFs.setVerifyChecksum(verifyChecksum); + super.setTimes(fullPath(f), mtime, atime); } @Override - public List> getDelegationTokens(String renewer) throws IOException { - return myFs.getDelegationTokens(renewer); + public Path resolvePath(final Path p) throws IOException { + return super.resolvePath(fullPath(p)); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index 0778e7f9eff..1addf2fd88b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -168,8 +168,7 @@ public class ViewFileSystem extends FileSystem { protected FileSystem getTargetFileSystem(final URI uri) throws URISyntaxException, IOException { - return new ChRootedFileSystem(FileSystem.get(uri, config), - new Path(uri.getPath())); + return new ChRootedFileSystem(uri, config); } @Override @@ -464,8 +463,11 @@ public class ViewFileSystem extends FileSystem { @Override public void setVerifyChecksum(final boolean verifyChecksum) { - // This is a file system level operations, however ViewFileSystem - // points to many file systems. Noop for ViewFileSystem. + List> mountPoints = + fsState.getMountPoints(); + for (InodeTree.MountPoint mount : mountPoints) { + mount.target.targetFileSystem.setVerifyChecksum(verifyChecksum); + } } public MountPoint[] getMountPoints() { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java index b2a9e16038d..05fec95631d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java @@ -72,14 +72,15 @@ public final class FileSystemTestHelper { public static String getAbsoluteTestRootDir(FileSystem fSys) throws IOException { - if (absTestRootDir == null) { + // NOTE: can't cache because of different filesystems! + //if (absTestRootDir == null) if (TEST_ROOT_DIR.startsWith("/")) { absTestRootDir = TEST_ROOT_DIR; } else { absTestRootDir = fSys.getWorkingDirectory().toString() + "/" + TEST_ROOT_DIR; } - } + //} return absTestRootDir; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java index 80347a72b45..0c24ad59b69 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java @@ -203,4 +203,58 @@ public class TestChecksumFileSystem { String str = readFile(localFs, testPath, 1024); assertEquals("testing stale checksum", str); } + + @Test + public void testRenameFileToFile() throws Exception { + Path srcPath = new Path(TEST_ROOT_DIR, "testRenameSrc"); + Path dstPath = new Path(TEST_ROOT_DIR, "testRenameDst"); + verifyRename(srcPath, dstPath, false); + } + + @Test + public void testRenameFileIntoDir() throws Exception { + Path srcPath = new Path(TEST_ROOT_DIR, "testRenameSrc"); + Path dstPath = new Path(TEST_ROOT_DIR, "testRenameDir"); + localFs.mkdirs(dstPath); + verifyRename(srcPath, dstPath, true); + } + + @Test + public void testRenameFileIntoDirFile() throws Exception { + Path srcPath = new Path(TEST_ROOT_DIR, "testRenameSrc"); + Path dstPath = new Path(TEST_ROOT_DIR, "testRenameDir/testRenameDst"); + assertTrue(localFs.mkdirs(dstPath)); + verifyRename(srcPath, dstPath, false); + } + + + void verifyRename(Path srcPath, Path dstPath, boolean dstIsDir) + throws Exception { + localFs.delete(srcPath,true); + localFs.delete(dstPath,true); + + Path realDstPath = dstPath; + if (dstIsDir) { + localFs.mkdirs(dstPath); + realDstPath = new Path(dstPath, srcPath.getName()); + } + + // ensure file + checksum are moved + writeFile(localFs, srcPath, 1); + assertTrue(localFs.exists(localFs.getChecksumFile(srcPath))); + assertTrue(localFs.rename(srcPath, dstPath)); + assertTrue(localFs.exists(localFs.getChecksumFile(realDstPath))); + + // create a file with no checksum, rename, ensure dst checksum is removed + writeFile(localFs.getRawFileSystem(), srcPath, 1); + assertFalse(localFs.exists(localFs.getChecksumFile(srcPath))); + assertTrue(localFs.rename(srcPath, dstPath)); + assertFalse(localFs.exists(localFs.getChecksumFile(realDstPath))); + + // create file with checksum, rename over prior dst with no checksum + writeFile(localFs, srcPath, 1); + assertTrue(localFs.exists(localFs.getChecksumFile(srcPath))); + assertTrue(localFs.rename(srcPath, dstPath)); + assertTrue(localFs.exists(localFs.getChecksumFile(realDstPath))); + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java index d1e9f598e66..c46ab96f375 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestChRootedFileSystem.java @@ -51,7 +51,7 @@ public class TestChRootedFileSystem { // ChRoot to the root of the testDirectory - fSys = new ChRootedFileSystem(fSysTarget, chrootedTo); + fSys = new ChRootedFileSystem(chrootedTo.toUri(), conf); } @After diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemDelegation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemDelegation.java new file mode 100644 index 00000000000..b9c0a087b31 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemDelegation.java @@ -0,0 +1,102 @@ +/** + * 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.fs.viewfs; + +import java.io.IOException; +import java.net.URI; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileSystemTestHelper; +import org.apache.hadoop.fs.FsConstants; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.*; +import static org.junit.Assert.*; + +/** + * Verify that viewfs propagates certain methods to the underlying fs + */ +public class TestViewFileSystemDelegation { //extends ViewFileSystemTestSetup { + static Configuration conf; + static FileSystem viewFs; + static FakeFileSystem fs1; + static FakeFileSystem fs2; + + @BeforeClass + public static void setup() throws Exception { + conf = ViewFileSystemTestSetup.configWithViewfsScheme(); + fs1 = setupFileSystem(new URI("fs1:/"), FakeFileSystem.class); + fs2 = setupFileSystem(new URI("fs2:/"), FakeFileSystem.class); + viewFs = FileSystem.get(FsConstants.VIEWFS_URI, conf); + } + + static FakeFileSystem setupFileSystem(URI uri, Class clazz) + throws Exception { + String scheme = uri.getScheme(); + conf.set("fs."+scheme+".impl", clazz.getName()); + FakeFileSystem fs = (FakeFileSystem)FileSystem.get(uri, conf); + assertEquals(uri, fs.getUri()); + Path targetPath = FileSystemTestHelper.getAbsoluteTestRootPath(fs); + ConfigUtil.addLink(conf, "/mounts/"+scheme, targetPath.toUri()); + return fs; + } + + @Test + public void testSanity() { + assertEquals("fs1:/", fs1.getUri().toString()); + assertEquals("fs2:/", fs2.getUri().toString()); + } + + @Test + public void testVerifyChecksum() throws Exception { + checkVerifyChecksum(false); + checkVerifyChecksum(true); + } + + void checkVerifyChecksum(boolean flag) { + viewFs.setVerifyChecksum(flag); + assertEquals(flag, fs1.getVerifyChecksum()); + assertEquals(flag, fs2.getVerifyChecksum()); + } + + static class FakeFileSystem extends LocalFileSystem { + boolean verifyChecksum = true; + URI uri; + + @Override + public void initialize(URI uri, Configuration conf) throws IOException { + super.initialize(uri, conf); + this.uri = uri; + } + + @Override + public URI getUri() { + return uri; + } + + @Override + public void setVerifyChecksum(boolean verifyChecksum) { + this.verifyChecksum = verifyChecksum; + } + + public boolean getVerifyChecksum(){ + return verifyChecksum; + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 6155736868e..529f40de243 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -208,6 +208,12 @@ Release 0.23.1 - UNRELEASED HDFS-2397. Undeprecate SecondaryNameNode (eli) + HDFS-2814. NamenodeMXBean does not account for svn revision in the version + information. (Hitesh Shah via jitendra) + + HDFS-2784. Update hftp and hdfs for host-based token support. + (Kihwal Lee via jitendra) + OPTIMIZATIONS HDFS-2130. Switch default checksum to CRC32C. (todd) @@ -231,6 +237,8 @@ Release 0.23.1 - UNRELEASED HDFS-2864. Remove some redundant methods and the constant METADATA_VERSION from FSDataset. (szetszwo) + HDFS-2879. Change FSDataset to package private. (szetszwo) + BUG FIXES HDFS-2541. For a sufficiently large value of blocks, the DN Scanner @@ -313,6 +321,9 @@ Release 0.23.1 - UNRELEASED HDFS-2835. Fix findbugs and javadoc issue with GetConf.java. (suresh) + HDFS-2889. getNumCurrentReplicas is package private but should be public on + 0.23 (see HDFS-2408). (Gregory Chanan via atm) + Release 0.23.0 - 2011-11-01 INCOMPATIBLE CHANGES @@ -1654,8 +1665,13 @@ Release 0.22.1 - Unreleased OPTIMIZATIONS + HDFS-2718. Optimize OP_ADD in edits loading. (shv) + BUG FIXES + HDFS-2877. If locking of a storage dir fails, it will remove the other + NN's lock file on exit. (todd) + Release 0.22.0 - 2011-11-29 INCOMPATIBLE CHANGES diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index e52ef995f05..be52b48f116 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -631,7 +631,7 @@ public class DFSClient implements java.io.Closeable { DelegationTokenIdentifier.stringifyToken(delToken)); ClientProtocol nn = DFSUtil.createNamenode - (NameNode.getAddress(token.getService().toString()), + (SecurityUtil.getTokenServiceAddr(delToken), conf, UserGroupInformation.getCurrentUser()); try { return nn.renewDelegationToken(delToken); @@ -649,7 +649,7 @@ public class DFSClient implements java.io.Closeable { LOG.info("Cancelling " + DelegationTokenIdentifier.stringifyToken(delToken)); ClientProtocol nn = DFSUtil.createNamenode( - NameNode.getAddress(token.getService().toString()), conf, + SecurityUtil.getTokenServiceAddr(delToken), conf, UserGroupInformation.getCurrentUser()); try { nn.cancelDelegationToken(delToken); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index f8b9ed45b69..ff81d7752b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -1538,7 +1538,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable { * write pipeline have failed. * @return the number of valid replicas of the current block */ - synchronized int getNumCurrentReplicas() throws IOException { + public synchronized int getNumCurrentReplicas() throws IOException { dfsClient.checkOpen(); isClosed(); if (streamer == null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 9dd6dfd37ed..119bca9b550 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -108,45 +108,10 @@ public class DistributedFileSystem extends FileSystem { InetSocketAddress namenode = NameNode.getAddress(uri.getAuthority()); this.dfs = new DFSClient(namenode, conf, statistics); - this.uri = URI.create(HdfsConstants.HDFS_URI_SCHEME + "://" + uri.getAuthority()); + this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority()); this.workingDir = getHomeDirectory(); } - /** Permit paths which explicitly specify the default port. */ - @Override - protected void checkPath(Path path) { - URI thisUri = this.getUri(); - URI thatUri = path.toUri(); - String thatAuthority = thatUri.getAuthority(); - if (thatUri.getScheme() != null - && thatUri.getScheme().equalsIgnoreCase(thisUri.getScheme()) - && thatUri.getPort() == NameNode.DEFAULT_PORT - && (thisUri.getPort() == -1 || - thisUri.getPort() == NameNode.DEFAULT_PORT) - && thatAuthority.substring(0,thatAuthority.indexOf(":")) - .equalsIgnoreCase(thisUri.getAuthority())) - return; - super.checkPath(path); - } - - /** Normalize paths that explicitly specify the default port. */ - @Override - public Path makeQualified(Path path) { - URI thisUri = this.getUri(); - URI thatUri = path.toUri(); - String thatAuthority = thatUri.getAuthority(); - if (thatUri.getScheme() != null - && thatUri.getScheme().equalsIgnoreCase(thisUri.getScheme()) - && thatUri.getPort() == NameNode.DEFAULT_PORT - && thisUri.getPort() == -1 - && thatAuthority.substring(0,thatAuthority.indexOf(":")) - .equalsIgnoreCase(thisUri.getAuthority())) { - path = new Path(thisUri.getScheme(), thisUri.getAuthority(), - thatUri.getPath()); - } - return super.makeQualified(path); - } - @Override public Path getWorkingDirectory() { return workingDir; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java index 7c56d780584..8fe8cba60d8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java @@ -59,6 +59,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenRenewer; +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector; import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.ServletUtil; import org.xml.sax.Attributes; @@ -89,17 +90,20 @@ public class HftpFileSystem extends FileSystem public static final Text TOKEN_KIND = new Text("HFTP delegation"); - private String nnHttpUrl; - private Text hdfsServiceName; + protected UserGroupInformation ugi; private URI hftpURI; + protected InetSocketAddress nnAddr; - protected UserGroupInformation ugi; + protected InetSocketAddress nnSecureAddr; public static final String HFTP_TIMEZONE = "UTC"; public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ"; + private Token delegationToken; private Token renewToken; - + private static final HftpDelegationTokenSelector hftpTokenSelector = + new HftpDelegationTokenSelector(); + public static final SimpleDateFormat getDateFormat() { final SimpleDateFormat df = new SimpleDateFormat(HFTP_DATE_FORMAT); df.setTimeZone(TimeZone.getTimeZone(HFTP_TIMEZONE)); @@ -115,11 +119,8 @@ public class HftpFileSystem extends FileSystem @Override protected int getDefaultPort() { - return getDefaultSecurePort(); - - //TODO: un-comment the following once HDFS-7510 is committed. -// return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, -// DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT); + return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, + DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT); } protected int getDefaultSecurePort() { @@ -127,16 +128,22 @@ public class HftpFileSystem extends FileSystem DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT); } - @Override - public String getCanonicalServiceName() { - return SecurityUtil.buildDTServiceName(hftpURI, getDefaultPort()); - } - - private String buildUri(String schema, String host, int port) { - StringBuilder sb = new StringBuilder(schema); - return sb.append(host).append(":").append(port).toString(); + protected InetSocketAddress getNamenodeAddr(URI uri) { + // use authority so user supplied uri can override port + return NetUtils.createSocketAddr(uri.getAuthority(), getDefaultPort()); } + protected InetSocketAddress getNamenodeSecureAddr(URI uri) { + // must only use the host and the configured https port + return NetUtils.createSocketAddrForHost(uri.getHost(), getDefaultSecurePort()); + } + + @Override + public String getCanonicalServiceName() { + // unlike other filesystems, hftp's service is the secure port, not the + // actual port in the uri + return SecurityUtil.buildTokenService(nnSecureAddr).toString(); + } @Override public void initialize(final URI name, final Configuration conf) @@ -144,95 +151,51 @@ public class HftpFileSystem extends FileSystem super.initialize(name, conf); setConf(conf); this.ugi = UserGroupInformation.getCurrentUser(); - nnAddr = NetUtils.createSocketAddr(name.toString()); - - // in case we open connection to hftp of a different cluster - // we need to know this cluster https port - // if it is not set we assume it is the same cluster or same port - int urlPort = conf.getInt("dfs.hftp.https.port", -1); - if(urlPort == -1) - urlPort = conf.getInt(DFSConfigKeys.DFS_HTTPS_PORT_KEY, - DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT); - - String normalizedNN = NetUtils.normalizeHostName(name.getHost()); - nnHttpUrl = buildUri("https://", normalizedNN ,urlPort); - LOG.debug("using url to get DT:" + nnHttpUrl); + this.nnAddr = getNamenodeAddr(name); + this.nnSecureAddr = getNamenodeSecureAddr(name); try { - hftpURI = new URI(buildUri("hftp://", normalizedNN, urlPort)); - } catch (URISyntaxException ue) { - throw new IOException("bad uri for hdfs", ue); - } - - // if one uses RPC port different from the Default one, - // one should specify what is the setvice name for this delegation token - // otherwise it is hostname:RPC_PORT - String key = DelegationTokenSelector.SERVICE_NAME_KEY - + SecurityUtil.buildDTServiceName(name, - DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT); - if(LOG.isDebugEnabled()) { - LOG.debug("Trying to find DT for " + name + " using key=" + key + - "; conf=" + conf.get(key, "")); - } - String nnServiceName = conf.get(key); - int nnPort = NameNode.DEFAULT_PORT; - if (nnServiceName != null) { // get the real port - nnPort = NetUtils.createSocketAddr(nnServiceName, - NameNode.DEFAULT_PORT).getPort(); - } - try { - URI hdfsURI = new URI("hdfs://" + normalizedNN + ":" + nnPort); - hdfsServiceName = new Text(SecurityUtil.buildDTServiceName(hdfsURI, - nnPort)); - } catch (URISyntaxException ue) { - throw new IOException("bad uri for hdfs", ue); + this.hftpURI = new URI(name.getScheme(), name.getAuthority(), + null, null, null); + } catch (URISyntaxException e) { + throw new IllegalArgumentException(e); } if (UserGroupInformation.isSecurityEnabled()) { - //try finding a token for this namenode (esp applicable for tasks - //using hftp). If there exists one, just set the delegationField - String hftpServiceName = getCanonicalServiceName(); - for (Token t : ugi.getTokens()) { - Text kind = t.getKind(); - if (DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(kind)) { - if (t.getService().equals(hdfsServiceName)) { - setDelegationToken(t); - break; - } - } else if (TOKEN_KIND.equals(kind)) { - if (hftpServiceName - .equals(normalizeService(t.getService().toString()))) { - setDelegationToken(t); - break; - } - } - } - - //since we don't already have a token, go get one over https - if (delegationToken == null) { - setDelegationToken(getDelegationToken(null)); + initDelegationToken(); + } + } + + protected void initDelegationToken() throws IOException { + // look for hftp token, then try hdfs + Token token = selectHftpDelegationToken(); + if (token == null) { + token = selectHdfsDelegationToken(); + } + + // if we don't already have a token, go get one over https + boolean createdToken = false; + if (token == null) { + token = getDelegationToken(null); + createdToken = (token != null); + } + + // we already had a token or getDelegationToken() didn't fail. + if (token != null) { + setDelegationToken(token); + if (createdToken) { dtRenewer.addRenewAction(this); + LOG.debug("Created new DT for " + token.getService()); + } else { + LOG.debug("Found existing DT for " + token.getService()); } } } - private String normalizeService(String service) { - int colonIndex = service.indexOf(':'); - if (colonIndex == -1) { - throw new IllegalArgumentException("Invalid service for hftp token: " + - service); - } - String hostname = - NetUtils.normalizeHostName(service.substring(0, colonIndex)); - String port = service.substring(colonIndex + 1); - return hostname + ":" + port; + protected Token selectHftpDelegationToken() { + Text serviceName = SecurityUtil.buildTokenService(nnSecureAddr); + return hftpTokenSelector.selectToken(serviceName, ugi.getTokens()); } - //TODO: un-comment the following once HDFS-7510 is committed. -// protected Token selectHftpDelegationToken() { -// Text serviceName = SecurityUtil.buildTokenService(nnSecureAddr); -// return hftpTokenSelector.selectToken(serviceName, ugi.getTokens()); -// } - protected Token selectHdfsDelegationToken() { return DelegationTokenSelector.selectHdfsDelegationToken( nnAddr, ugi, getConf()); @@ -245,13 +208,17 @@ public class HftpFileSystem extends FileSystem } @Override - public void setDelegationToken(Token token) { + public synchronized void setDelegationToken(Token token) { renewToken = token; // emulate the 203 usage of the tokens // by setting the kind and service as if they were hdfs tokens delegationToken = new Token(token); + // NOTE: the remote nn must be configured to use hdfs delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND); - delegationToken.setService(hdfsServiceName); + // no need to change service because we aren't exactly sure what it + // should be. we can guess, but it might be wrong if the local conf + // value is incorrect. the service is a client side field, so the remote + // end does not care about the value } @Override @@ -262,6 +229,7 @@ public class HftpFileSystem extends FileSystem ugi.reloginFromKeytab(); return ugi.doAs(new PrivilegedExceptionAction>() { public Token run() throws IOException { + final String nnHttpUrl = DFSUtil.createUri("https", nnSecureAddr).toString(); Credentials c; try { c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer); @@ -291,12 +259,7 @@ public class HftpFileSystem extends FileSystem @Override public URI getUri() { - try { - return new URI("hftp", null, nnAddr.getHostName(), nnAddr.getPort(), - null, null, null); - } catch (URISyntaxException e) { - return null; - } + return hftpURI; } /** @@ -722,11 +685,12 @@ public class HftpFileSystem extends FileSystem public long renew(Token token, Configuration conf) throws IOException { // update the kerberos credentials, if they are coming from a keytab - UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab(); + UserGroupInformation.getLoginUser().reloginFromKeytab(); // use https to renew the token + InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token); return DelegationTokenFetcher.renewDelegationToken - ("https://" + token.getService().toString(), + (DFSUtil.createUri("https", serviceAddr).toString(), (Token) token); } @@ -737,10 +701,18 @@ public class HftpFileSystem extends FileSystem // update the kerberos credentials, if they are coming from a keytab UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab(); // use https to cancel the token + InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token); DelegationTokenFetcher.cancelDelegationToken - ("https://" + token.getService().toString(), + (DFSUtil.createUri("https", serviceAddr).toString(), (Token) token); + } + } + + private static class HftpDelegationTokenSelector + extends AbstractDelegationTokenSelector { + + public HftpDelegationTokenSelector() { + super(TOKEN_KIND); } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java index fa1a89f4b96..97e3b2414a3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs; import java.io.FileInputStream; import java.io.IOException; import java.net.HttpURLConnection; +import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; import java.net.URL; @@ -120,6 +121,16 @@ public class HsftpFileSystem extends HftpFileSystem { } } + @Override + protected int getDefaultPort() { + return getDefaultSecurePort(); + } + + @Override + protected InetSocketAddress getNamenodeSecureAddr(URI uri) { + return getNamenodeAddr(uri); + } + @Override protected HttpURLConnection openConnection(String path, String query) throws IOException { @@ -161,16 +172,6 @@ public class HsftpFileSystem extends HftpFileSystem { return (HttpURLConnection) conn; } - @Override - public URI getUri() { - try { - return new URI("hsftp", null, nnAddr.getHostName(), nnAddr.getPort(), - null, null, null); - } catch (URISyntaxException e) { - return null; - } - } - /** * Dummy hostname verifier that is used to bypass hostname checking */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java index 8f8ef8e0676..b5f24d18557 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; @@ -296,8 +297,7 @@ public class DelegationTokenSecretManager } final InetSocketAddress addr = namenode.getNameNodeAddress(); - final String s = addr.getAddress().getHostAddress() + ":" + addr.getPort(); - token.setService(new Text(s)); + SecurityUtil.setTokenService(token, addr); final Credentials c = new Credentials(); c.addToken(new Text(ugi.getShortUserName()), token); return c; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java index f2f47a251fc..beeb2c8fa21 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java @@ -157,9 +157,6 @@ public class BlockInfoUnderConstruction extends BlockInfo { BlockInfo convertToCompleteBlock() throws IOException { assert getBlockUCState() != BlockUCState.COMPLETE : "Trying to convert a COMPLETE block"; - if(getBlockUCState() != BlockUCState.COMMITTED) - throw new IOException( - "Cannot complete block: block has not been COMMITTED by the client"); return new BlockInfo(this); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index ee94414335a..1aa22796ef5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -441,15 +441,23 @@ public class BlockManager { */ private BlockInfo completeBlock(final INodeFile fileINode, final int blkIndex) throws IOException { + return completeBlock(fileINode, blkIndex, false); + } + + public BlockInfo completeBlock(final INodeFile fileINode, + final int blkIndex, final boolean force) throws IOException { if(blkIndex < 0) return null; BlockInfo curBlock = fileINode.getBlocks()[blkIndex]; if(curBlock.isComplete()) return curBlock; BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock; - if(ucBlock.numNodes() < minReplication) + if(!force && ucBlock.numNodes() < minReplication) throw new IOException("Cannot complete block: " + "block does not satisfy minimal replication requirement."); + if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED) + throw new IOException( + "Cannot complete block: block has not been COMMITTED by the client"); BlockInfo completeBlock = ucBlock.convertToCompleteBlock(); // replace penultimate block in file fileINode.setBlock(blkIndex, completeBlock); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java index b20e694acf4..bfc7e355cf1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java @@ -62,6 +62,7 @@ import org.apache.hadoop.http.HtmlQuoting; import org.apache.hadoop.io.Text; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.AccessControlException; +import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod; import org.apache.hadoop.security.authentication.util.KerberosName; @@ -492,7 +493,7 @@ public class JspHelper { return UserGroupInformation.createRemoteUser(strings[0]); } - private static String getNNServiceAddress(ServletContext context, + private static InetSocketAddress getNNServiceAddress(ServletContext context, HttpServletRequest request) { String namenodeAddressInUrl = request.getParameter(NAMENODE_ADDRESS); InetSocketAddress namenodeAddress = null; @@ -503,8 +504,7 @@ public class JspHelper { context); } if (namenodeAddress != null) { - return (namenodeAddress.getAddress().getHostAddress() + ":" - + namenodeAddress.getPort()); + return namenodeAddress; } return null; } @@ -547,9 +547,9 @@ public class JspHelper { Token token = new Token(); token.decodeFromUrlString(tokenString); - String serviceAddress = getNNServiceAddress(context, request); + InetSocketAddress serviceAddress = getNNServiceAddress(context, request); if (serviceAddress != null) { - token.setService(new Text(serviceAddress)); + SecurityUtil.setTokenService(token, serviceAddress); token.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND); } ByteArrayInputStream buf = new ByteArrayInputStream(token diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java index 4c11973d4ae..3de906701fd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java @@ -599,8 +599,12 @@ public abstract class Storage extends StorageInfo { * @throws IOException if locking fails. */ FileLock tryLock() throws IOException { + boolean deletionHookAdded = false; File lockF = new File(root, STORAGE_FILE_LOCK); - lockF.deleteOnExit(); + if (!lockF.exists()) { + lockF.deleteOnExit(); + deletionHookAdded = true; + } RandomAccessFile file = new RandomAccessFile(lockF, "rws"); FileLock res = null; try { @@ -613,6 +617,12 @@ public abstract class Storage extends StorageInfo { file.close(); throw e; } + if (res != null && !deletionHookAdded) { + // If the file existed prior to our startup, we didn't + // call deleteOnExit above. But since we successfully locked + // the dir, we can take care of cleaning it up. + lockF.deleteOnExit(); + } return res; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index 61bc29acf46..153fd93854a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -222,7 +222,7 @@ class BlockReceiver implements Closeable { cleanupBlock(); // check if there is a disk error - IOException cause = FSDataset.getCauseIfDiskError(ioe); + IOException cause = DatanodeUtil.getCauseIfDiskError(ioe); DataNode.LOG.warn("IOException in BlockReceiver constructor. Cause is ", cause); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java new file mode 100644 index 00000000000..cb0767c93a2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java @@ -0,0 +1,67 @@ +/** + * 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.hdfs.server.datanode; + +import java.io.File; +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdfs.protocol.Block; + +/** Provide utility methods for Datanode. */ +@InterfaceAudience.Private +class DatanodeUtil { + private final static String DISK_ERROR = "Possible disk error on file creation: "; + + /** Get the cause of an I/O exception if caused by a possible disk error + * @param ioe an I/O exception + * @return cause if the I/O exception is caused by a possible disk error; + * null otherwise. + */ + static IOException getCauseIfDiskError(IOException ioe) { + if (ioe.getMessage()!=null && ioe.getMessage().startsWith(DISK_ERROR)) { + return (IOException)ioe.getCause(); + } else { + return null; + } + } + + /** + * Create a new file. + * @throws IOException + * if the file already exists or if the file cannot be created. + */ + static File createTmpFile(Block b, File f) throws IOException { + if (f.exists()) { + throw new IOException("Unexpected problem in creating temporary file for " + + b + ". File " + f + " should not be present, but is."); + } + // Create the zero-length temp file + final boolean fileCreated; + try { + fileCreated = f.createNewFile(); + } catch (IOException ioe) { + throw (IOException)new IOException(DISK_ERROR + f).initCause(ioe); + } + if (!fileCreated) { + throw new IOException("Unexpected problem in creating temporary file for " + + b + ". File " + f + " should be creatable, but is already present."); + } + return f; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java index 1c362dd1bd0..7247a8e4837 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java @@ -75,7 +75,7 @@ import org.apache.hadoop.util.ReflectionUtils; * ***************************************************/ @InterfaceAudience.Private -public class FSDataset implements FSDatasetInterface { +class FSDataset implements FSDatasetInterface { /** * A node type that can be built into a tree reflecting the @@ -373,7 +373,7 @@ public class FSDataset implements FSDatasetInterface { */ File createTmpFile(Block b) throws IOException { File f = new File(tmpDir, b.getBlockName()); - return FSDataset.createTmpFile(b, f); + return DatanodeUtil.createTmpFile(b, f); } /** @@ -382,7 +382,7 @@ public class FSDataset implements FSDatasetInterface { */ File createRbwFile(Block b) throws IOException { File f = new File(rbwDir, b.getBlockName()); - return FSDataset.createTmpFile(b, f); + return DatanodeUtil.createTmpFile(b, f); } File addBlock(Block b, File f) throws IOException { @@ -536,15 +536,15 @@ public class FSDataset implements FSDatasetInterface { } /** Return storage directory corresponding to the volume */ - public File getDir() { + File getDir() { return currentDir.getParentFile(); } - public File getCurrentDir() { + File getCurrentDir() { return currentDir; } - public File getRbwDir(String bpid) throws IOException { + File getRbwDir(String bpid) throws IOException { BlockPoolSlice bp = getBlockPoolSlice(bpid); return bp.getRbwDir(); } @@ -1085,26 +1085,6 @@ public class FSDataset implements FSDatasetInterface { return new MetaDataInputStream(new FileInputStream(checksumFile), checksumFile.length()); } - - static File createTmpFile(Block b, File f) throws IOException { - if (f.exists()) { - throw new IOException("Unexpected problem in creating temporary file for "+ - b + ". File " + f + " should not be present, but is."); - } - // Create the zero-length temp file - // - boolean fileCreated = false; - try { - fileCreated = f.createNewFile(); - } catch (IOException ioe) { - throw (IOException)new IOException(DISK_ERROR +f).initCause(ioe); - } - if (!fileCreated) { - throw new IOException("Unexpected problem in creating temporary file for "+ - b + ". File " + f + " should be creatable, but is already present."); - } - return f; - } private final DataNode datanode; final FSVolumeSet volumes; @@ -1246,7 +1226,7 @@ public class FSDataset implements FSDatasetInterface { /** * Get File name for a given block. */ - public File getBlockFile(ExtendedBlock b) throws IOException { + private File getBlockFile(ExtendedBlock b) throws IOException { return getBlockFile(b.getBlockPoolId(), b.getLocalBlock()); } @@ -1320,7 +1300,7 @@ public class FSDataset implements FSDatasetInterface { * @throws ReplicaNotFoundException if no entry is in the map or * there is a generation stamp mismatch */ - private ReplicaInfo getReplicaInfo(ExtendedBlock b) + ReplicaInfo getReplicaInfo(ExtendedBlock b) throws ReplicaNotFoundException { ReplicaInfo info = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock()); if (info == null) { @@ -1456,19 +1436,6 @@ public class FSDataset implements FSDatasetInterface { } } - private final static String DISK_ERROR = "Possible disk error on file creation: "; - /** Get the cause of an I/O exception if caused by a possible disk error - * @param ioe an I/O exception - * @return cause if the I/O exception is caused by a possible disk error; - * null otherwise. - */ - static IOException getCauseIfDiskError(IOException ioe) { - if (ioe.getMessage()!=null && ioe.getMessage().startsWith(DISK_ERROR)) { - return (IOException)ioe.getCause(); - } else { - return null; - } - } @Override // FSDatasetInterface public synchronized ReplicaInPipelineInterface append(ExtendedBlock b, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java index a29f0a959d7..3adc0ccc8a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java @@ -162,7 +162,7 @@ abstract public class ReplicaInfo extends Block implements Replica { * be recovered (especially on Windows) on datanode restart. */ private void unlinkFile(File file, Block b) throws IOException { - File tmpFile = FSDataset.createTmpFile(b, FSDataset.getUnlinkTmpFile(file)); + File tmpFile = DatanodeUtil.createTmpFile(b, FSDataset.getUnlinkTmpFile(file)); try { FileInputStream in = new FileInputStream(file); try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index 49b584630ad..ca226e3cc91 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -262,22 +262,28 @@ public class FSDirectory implements Closeable { short replication, long modificationTime, long atime, - long preferredBlockSize) + long preferredBlockSize, + String clientName, + String clientMachine) throws UnresolvedLinkException { INode newNode; - long diskspace = UNKNOWN_DISK_SPACE; assert hasWriteLock(); if (blocks == null) newNode = new INodeDirectory(permissions, modificationTime); - else { + else if(blocks.length == 0 || blocks[blocks.length-1].getBlockUCState() + == BlockUCState.UNDER_CONSTRUCTION) { + newNode = new INodeFileUnderConstruction( + permissions, blocks.length, replication, + preferredBlockSize, modificationTime, clientName, + clientMachine, null); + } else { newNode = new INodeFile(permissions, blocks.length, replication, modificationTime, atime, preferredBlockSize); - diskspace = ((INodeFile)newNode).diskspaceConsumed(blocks); } writeLock(); try { try { - newNode = addNode(path, newNode, diskspace); + newNode = addNode(path, newNode, UNKNOWN_DISK_SPACE); if(newNode != null && blocks != null) { int nrBlocks = blocks.length; // Add file->block mapping @@ -296,6 +302,74 @@ public class FSDirectory implements Closeable { } + /** + * Update files in-memory data structures with new block information. + * @throws IOException + */ + void updateFile(INodeFile file, + String path, + PermissionStatus permissions, + BlockInfo[] blocks, + short replication, + long mtime, + long atime, + long preferredBlockSize) throws IOException { + + // Update the salient file attributes. + file.setAccessTime(atime); + file.setModificationTimeForce(mtime); + + // Update its block list + BlockInfo[] oldBlocks = file.getBlocks(); + + // Are we only updating the last block's gen stamp. + boolean isGenStampUpdate = oldBlocks.length == blocks.length; + + // First, update blocks in common + BlockInfo oldBlock = null; + for (int i = 0; i < oldBlocks.length && i < blocks.length; i++) { + oldBlock = oldBlocks[i]; + Block newBlock = blocks[i]; + + boolean isLastBlock = i == oldBlocks.length - 1; + if (oldBlock.getBlockId() != newBlock.getBlockId() || + (oldBlock.getGenerationStamp() != newBlock.getGenerationStamp() && + !(isGenStampUpdate && isLastBlock))) { + throw new IOException("Mismatched block IDs or generation stamps, " + + "attempting to replace block " + oldBlock + " with " + newBlock + + " as block # " + i + "/" + blocks.length + " of " + path); + } + + oldBlock.setNumBytes(newBlock.getNumBytes()); + oldBlock.setGenerationStamp(newBlock.getGenerationStamp()); + } + + if (blocks.length < oldBlocks.length) { + // We're removing a block from the file, e.g. abandonBlock(...) + if (!file.isUnderConstruction()) { + throw new IOException("Trying to remove a block from file " + + path + " which is not under construction."); + } + if (blocks.length != oldBlocks.length - 1) { + throw new IOException("Trying to remove more than one block from file " + + path); + } + unprotectedRemoveBlock(path, + (INodeFileUnderConstruction)file, oldBlocks[oldBlocks.length - 1]); + } else if (blocks.length > oldBlocks.length) { + // We're adding blocks + // First complete last old Block + getBlockManager().completeBlock(file, oldBlocks.length-1, true); + // Add the new blocks + for (int i = oldBlocks.length; i < blocks.length; i++) { + // addBlock(); + BlockInfo newBI = blocks[i]; + getBlockManager().addINode(newBI, file); + file.addBlock(newBI); + } + } + } + INodeDirectory addToParent(byte[] src, INodeDirectory parentINode, INode newNode, boolean propagateModTime) throws UnresolvedLinkException { // NOTE: This does not update space counts for parents @@ -417,28 +491,33 @@ public class FSDirectory implements Closeable { writeLock(); try { - // modify file-> block and blocksMap - fileNode.removeLastBlock(block); - getBlockManager().removeBlockFromMap(block); - + unprotectedRemoveBlock(path, fileNode, block); // write modified block locations to log fsImage.getEditLog().logOpenFile(path, fileNode); - if(NameNode.stateChangeLog.isDebugEnabled()) { - NameNode.stateChangeLog.debug("DIR* FSDirectory.removeBlock: " - +path+" with "+block - +" block is removed from the file system"); - } - - // update space consumed - INode[] pathINodes = getExistingPathINodes(path); - updateCount(pathINodes, pathINodes.length-1, 0, - -fileNode.getPreferredBlockSize()*fileNode.getReplication(), true); } finally { writeUnlock(); } return true; } + void unprotectedRemoveBlock(String path, INodeFileUnderConstruction fileNode, + Block block) throws IOException { + // modify file-> block and blocksMap + fileNode.removeLastBlock(block); + getBlockManager().removeBlockFromMap(block); + + if(NameNode.stateChangeLog.isDebugEnabled()) { + NameNode.stateChangeLog.debug("DIR* FSDirectory.removeBlock: " + +path+" with "+block + +" block is removed from the file system"); + } + + // update space consumed + INode[] pathINodes = getExistingPathINodes(path); + updateCount(pathINodes, pathINodes.length-1, 0, + -fileNode.getPreferredBlockSize()*fileNode.getReplication(), true); + } + /** * @see #unprotectedRenameTo(String, String, long) * @deprecated Use {@link #renameTo(String, String, Rename...)} instead. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java index 991fd08c842..410df0a0fa1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java @@ -187,31 +187,53 @@ public class FSEditLogLoader { " clientMachine " + addCloseOp.clientMachine); } - fsDir.unprotectedDelete(addCloseOp.path, addCloseOp.mtime); + // There are four cases here: + // 1. OP_ADD to create a new file + // 2. OP_ADD to update file blocks + // 3. OP_ADD to open file for append + // 4. OP_CLOSE to close the file - // add to the file tree - INodeFile node = (INodeFile)fsDir.unprotectedAddFile( - addCloseOp.path, permissions, - blocks, replication, - addCloseOp.mtime, addCloseOp.atime, blockSize); - if (addCloseOp.opCode == FSEditLogOpCodes.OP_ADD) { - // - // Replace current node with a INodeUnderConstruction. - // Recreate in-memory lease record. - // - INodeFileUnderConstruction cons = new INodeFileUnderConstruction( - node.getLocalNameBytes(), - node.getReplication(), - node.getModificationTime(), - node.getPreferredBlockSize(), - node.getBlocks(), - node.getPermissionStatus(), - addCloseOp.clientName, - addCloseOp.clientMachine, - null); - fsDir.replaceNode(addCloseOp.path, node, cons); - fsNamesys.leaseManager.addLease(cons.getClientName(), - addCloseOp.path); + // See if the file already exists + INodeFile oldFile = fsDir.getFileINode(addCloseOp.path); + if (oldFile == null) { // OP_ADD for a new file + assert addCloseOp.opCode == FSEditLogOpCodes.OP_ADD : + "Expected opcode OP_ADD, but got " + addCloseOp.opCode; + fsDir.unprotectedAddFile( + addCloseOp.path, permissions, blocks, replication, + addCloseOp.mtime, addCloseOp.atime, blockSize, + addCloseOp.clientName, addCloseOp.clientMachine); + } else { + fsDir.updateFile(oldFile, + addCloseOp.path, permissions, blocks, replication, + addCloseOp.mtime, addCloseOp.atime, blockSize); + if(addCloseOp.opCode == FSEditLogOpCodes.OP_CLOSE) { // OP_CLOSE + assert oldFile.isUnderConstruction() : + "File is not under construction: " + addCloseOp.path; + fsNamesys.getBlockManager().completeBlock( + oldFile, blocks.length-1, true); + INodeFile newFile = + ((INodeFileUnderConstruction)oldFile).convertToInodeFile(); + fsDir.replaceNode(addCloseOp.path, oldFile, newFile); + } else if(! oldFile.isUnderConstruction()) { // OP_ADD for append + INodeFileUnderConstruction cons = new INodeFileUnderConstruction( + oldFile.getLocalNameBytes(), + oldFile.getReplication(), + oldFile.getModificationTime(), + oldFile.getPreferredBlockSize(), + oldFile.getBlocks(), + oldFile.getPermissionStatus(), + addCloseOp.clientName, + addCloseOp.clientMachine, + null); + fsDir.replaceNode(addCloseOp.path, oldFile, cons); + } + } + // Update file lease + if(addCloseOp.opCode == FSEditLogOpCodes.OP_ADD) { + fsNamesys.leaseManager.addLease(addCloseOp.clientName, addCloseOp.path); + } else { // Ops.OP_CLOSE + fsNamesys.leaseManager.removeLease( + ((INodeFileUnderConstruction)oldFile).getClientName(), addCloseOp.path); } break; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index ae107bd669c..e4f03386456 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -4249,7 +4249,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats, */ @Override // NameNodeMXBean public String getVersion() { - return VersionInfo.getVersion(); + return VersionInfo.getVersion() + ", r" + VersionInfo.getRevision(); } @Override // NameNodeMXBean diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java index 2440c4dd122..0fab53c95ff 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java @@ -41,8 +41,20 @@ public class INodeFileUnderConstruction extends INodeFile { String clientName, String clientMachine, DatanodeDescriptor clientNode) { - super(permissions.applyUMask(UMASK), 0, replication, modTime, modTime, - preferredBlockSize); + this(permissions, 0, replication, preferredBlockSize, modTime, + clientName, clientMachine, clientNode); + } + + INodeFileUnderConstruction(PermissionStatus permissions, + int nrBlocks, + short replication, + long preferredBlockSize, + long modTime, + String clientName, + String clientMachine, + DatanodeDescriptor clientNode) { + super(permissions.applyUMask(UMASK), nrBlocks, replication, + modTime, modTime, preferredBlockSize); this.clientName = clientName; this.clientMachine = clientMachine; this.clientNode = clientNode; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java index b984e9189cf..0d062959c0b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java @@ -25,6 +25,7 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.io.PrintStream; import java.net.HttpURLConnection; +import java.net.InetSocketAddress; import java.net.URL; import java.net.URLConnection; import java.security.PrivilegedExceptionAction; @@ -49,6 +50,7 @@ import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet; import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; +import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; @@ -201,7 +203,8 @@ public class DelegationTokenFetcher { static public Credentials getDTfromRemote(String nnAddr, String renewer) throws IOException { DataInputStream dis = null; - + InetSocketAddress serviceAddr = NetUtils.createSocketAddr(nnAddr); + try { StringBuffer url = new StringBuffer(); if (renewer != null) { @@ -221,9 +224,7 @@ public class DelegationTokenFetcher { ts.readFields(dis); for(Token token: ts.getAllTokens()) { token.setKind(HftpFileSystem.TOKEN_KIND); - token.setService(new Text(SecurityUtil.buildDTServiceName - (remoteURL.toURI(), - DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT))); + SecurityUtil.setTokenService(token, serviceAddr); } return ts; } catch (Exception e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index db8186b8d13..aeba7e03f47 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -882,6 +882,8 @@ public class MiniDFSCluster { if(dn == null) throw new IOException("Cannot start DataNode in " + dnConf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY)); + //NOTE: the following is true if and only if: + // hadoop.security.token.service.use_ip=true //since the HDFS does things based on IP:port, we need to add the mapping //for IP:port to rackId String ipAddr = dn.getSelfAddr().getAddress().getHostAddress(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java index 1613e82ca2d..582767c8010 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java @@ -72,12 +72,20 @@ public class TestAbandonBlock { // Now abandon the last block DFSClient dfsclient = DFSClientAdapter.getDFSClient((DistributedFileSystem)fs); - LocatedBlocks blocks = dfsclient.getNamenode().getBlockLocations(src, 0, 1); + LocatedBlocks blocks = + dfsclient.getNamenode().getBlockLocations(src, 0, Integer.MAX_VALUE); + int orginalNumBlocks = blocks.locatedBlockCount(); LocatedBlock b = blocks.getLastLocatedBlock(); dfsclient.getNamenode().abandonBlock(b.getBlock(), src, dfsclient.clientName); // And close the file fout.close(); + + // Close cluster and check the block has been abandoned after restart + cluster.restartNameNode(); + blocks = dfsclient.getNamenode().getBlockLocations(src, 0, Integer.MAX_VALUE); + assert orginalNumBlocks == blocks.locatedBlockCount() + 1 : + "Blocks " + b + " has not been abandoned."; } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java index 97387ccfc14..55e744e1a0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs; import java.io.ByteArrayOutputStream; +import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintStream; @@ -35,16 +36,15 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.server.datanode.FSDataset; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck; import org.apache.hadoop.hdfs.tools.DFSck; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.util.ToolRunner; - -import org.junit.Before; import org.junit.After; +import org.junit.Assert; +import org.junit.Before; import org.junit.Test; -import junit.framework.Assert; /** * Class is used to test client reporting corrupted block replica to name node. @@ -305,9 +305,9 @@ public class TestClientReportBadBlock { */ private static void corruptBlock(final ExtendedBlock block, final DataNode dn) throws FileNotFoundException, IOException { - final FSDataset data = (FSDataset) dn.getFSDataset(); - final RandomAccessFile raFile = new RandomAccessFile( - data.getBlockFile(block), "rw"); + final File f = DataNodeTestUtils.getBlockFile( + dn, block.getBlockPoolId(), block.getLocalBlock()); + final RandomAccessFile raFile = new RandomAccessFile(f, "rw"); final byte[] bytes = new byte[(int) BLOCK_SIZE]; for (int i = 0; i < BLOCK_SIZE; i++) { bytes[i] = (byte) (i); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java index 44236966116..e5cf21fea5b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java @@ -46,7 +46,6 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; -import org.apache.hadoop.hdfs.server.datanode.FSDataset; import org.apache.hadoop.hdfs.tools.DFSAdmin; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; @@ -1164,9 +1163,9 @@ public class TestDFSShell extends TestCase { String poolId = cluster.getNamesystem().getBlockPoolId(); Iterable[] blocks = cluster.getAllBlockReports(poolId); for(int i = 0; i < blocks.length; i++) { - FSDataset ds = (FSDataset)datanodes.get(i).getFSDataset(); + DataNode dn = datanodes.get(i); for(Block b : blocks[i]) { - files.add(DataNodeTestUtils.getBlockFile(ds, poolId, b.getBlockId())); + files.add(DataNodeTestUtils.getFile(dn, poolId, b.getBlockId())); } } return files; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java index 2f65c1a49d3..d8b58f7cb6c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java @@ -17,29 +17,27 @@ */ package org.apache.hadoop.hdfs; -import static org.junit.Assert.*; +import static org.junit.Assert.assertTrue; import java.io.File; -import java.io.IOException; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.InetSocketAddress; import java.util.List; -import org.junit.Test; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.HardLink; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; -import org.apache.hadoop.hdfs.server.datanode.FSDataset; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.junit.Test; /** * This class tests the building blocks that are needed to @@ -133,14 +131,13 @@ public class TestFileAppend{ LocatedBlocks locations = client.getNamenode().getBlockLocations( file1.toString(), 0, Long.MAX_VALUE); List blocks = locations.getLocatedBlocks(); - FSDataset dataset = (FSDataset) dn[0].data; // // Create hard links for a few of the blocks // for (int i = 0; i < blocks.size(); i = i + 2) { ExtendedBlock b = blocks.get(i).getBlock(); - final File f = DataNodeTestUtils.getBlockFile(dataset, + final File f = DataNodeTestUtils.getFile(dn[0], b.getBlockPoolId(), b.getLocalBlock().getBlockId()); File link = new File(f.toString() + ".link"); System.out.println("Creating hardlink for File " + f + " to " + link); @@ -154,7 +151,7 @@ public class TestFileAppend{ ExtendedBlock b = blocks.get(i).getBlock(); System.out.println("testCopyOnWrite detaching block " + b); assertTrue("Detaching block " + b + " should have returned true", - dataset.unlinkBlock(b, 1)); + DataNodeTestUtils.unlinkBlock(dn[0], b, 1)); } // Since the blocks were already detached earlier, these calls should @@ -164,7 +161,7 @@ public class TestFileAppend{ ExtendedBlock b = blocks.get(i).getBlock(); System.out.println("testCopyOnWrite detaching block " + b); assertTrue("Detaching block " + b + " should have returned false", - !dataset.unlinkBlock(b, 1)); + !DataNodeTestUtils.unlinkBlock(dn[0], b, 1)); } } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java index b6f610937f0..03b731fdb22 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs; +import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; @@ -35,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.hdfs.server.datanode.FSDataset; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.LeaseManager; import org.apache.hadoop.hdfs.server.namenode.NameNode; @@ -199,8 +200,9 @@ public class TestFileAppend3 extends junit.framework.TestCase { DatanodeInfo[] datanodeinfos = lb.getLocations(); assertEquals(repl, datanodeinfos.length); final DataNode dn = cluster.getDataNode(datanodeinfos[0].getIpcPort()); - final FSDataset data = (FSDataset)dn.getFSDataset(); - final RandomAccessFile raf = new RandomAccessFile(data.getBlockFile(blk), "rw"); + final File f = DataNodeTestUtils.getBlockFile( + dn, blk.getBlockPoolId(), blk.getLocalBlock()); + final RandomAccessFile raf = new RandomAccessFile(f, "rw"); AppendTestUtil.LOG.info("dn=" + dn + ", blk=" + blk + " (length=" + blk.getNumBytes() + ")"); assertEquals(len1, raf.length()); raf.setLength(0); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java index 94614c15f9a..ed1508a2680 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java @@ -17,6 +17,21 @@ */ package org.apache.hadoop.hdfs; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY; + import java.io.BufferedReader; import java.io.File; import java.io.FileNotFoundException; @@ -36,7 +51,6 @@ import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.ParentNotDirectoryException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; -import static org.apache.hadoop.hdfs.DFSConfigKeys.*; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -45,7 +59,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; -import org.apache.hadoop.hdfs.server.datanode.FSDataset; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.LeaseManager; @@ -829,10 +842,9 @@ public class TestFileCreation extends junit.framework.TestCase { int successcount = 0; for(DatanodeInfo datanodeinfo: locatedblock.getLocations()) { DataNode datanode = cluster.getDataNode(datanodeinfo.ipcPort); - FSDataset dataset = (FSDataset)datanode.data; ExtendedBlock blk = locatedblock.getBlock(); - Block b = dataset.getStoredBlock(blk.getBlockPoolId(), blk.getBlockId()); - final File blockfile = DataNodeTestUtils.getBlockFile(dataset, + Block b = datanode.data.getStoredBlock(blk.getBlockPoolId(), blk.getBlockId()); + final File blockfile = DataNodeTestUtils.getFile(datanode, blk.getBlockPoolId(), b.getBlockId()); System.out.println("blockfile=" + blockfile); if (blockfile != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpFileSystem.java index ac639bd939b..3b617c71d51 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpFileSystem.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs; import java.io.IOException; import java.net.URISyntaxException; +import java.net.URI; import java.net.URL; import java.net.HttpURLConnection; import java.util.Random; @@ -232,4 +233,164 @@ public class TestHftpFileSystem { in.seek(7); assertEquals('7', in.read()); } + + public void resetFileSystem() throws IOException { + // filesystem caching has a quirk/bug that it caches based on the user's + // given uri. the result is if a filesystem is instantiated with no port, + // it gets the default port. then if the default port is changed, + // and another filesystem is instantiated with no port, the prior fs + // is returned, not a new one using the changed port. so let's flush + // the cache between tests... + FileSystem.closeAll(); + } + + @Test + public void testHftpDefaultPorts() throws IOException { + resetFileSystem(); + Configuration conf = new Configuration(); + URI uri = URI.create("hftp://localhost"); + HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf); + + assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, fs.getDefaultPort()); + assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultSecurePort()); + + assertEquals(uri, fs.getUri()); + assertEquals( + "127.0.0.1:"+DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, + fs.getCanonicalServiceName() + ); + } + + @Test + public void testHftpCustomDefaultPorts() throws IOException { + resetFileSystem(); + Configuration conf = new Configuration(); + conf.setInt("dfs.http.port", 123); + conf.setInt("dfs.https.port", 456); + + URI uri = URI.create("hftp://localhost"); + HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf); + + assertEquals(123, fs.getDefaultPort()); + assertEquals(456, fs.getDefaultSecurePort()); + + assertEquals(uri, fs.getUri()); + assertEquals( + "127.0.0.1:456", + fs.getCanonicalServiceName() + ); + } + + @Test + public void testHftpCustomUriPortWithDefaultPorts() throws IOException { + resetFileSystem(); + Configuration conf = new Configuration(); + URI uri = URI.create("hftp://localhost:123"); + HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf); + + assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, fs.getDefaultPort()); + assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultSecurePort()); + + assertEquals(uri, fs.getUri()); + assertEquals( + "127.0.0.1:"+DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, + fs.getCanonicalServiceName() + ); + } + + @Test + public void testHftpCustomUriPortWithCustomDefaultPorts() throws IOException { + resetFileSystem(); + Configuration conf = new Configuration(); + conf.setInt("dfs.http.port", 123); + conf.setInt("dfs.https.port", 456); + + URI uri = URI.create("hftp://localhost:789"); + HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf); + + assertEquals(123, fs.getDefaultPort()); + assertEquals(456, fs.getDefaultSecurePort()); + + assertEquals(uri, fs.getUri()); + assertEquals( + "127.0.0.1:456", + fs.getCanonicalServiceName() + ); + } + + /// + + @Test + public void testHsftpDefaultPorts() throws IOException { + resetFileSystem(); + Configuration conf = new Configuration(); + URI uri = URI.create("hsftp://localhost"); + HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf); + + assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultPort()); + assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultSecurePort()); + + assertEquals(uri, fs.getUri()); + assertEquals( + "127.0.0.1:"+DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, + fs.getCanonicalServiceName() + ); + } + + @Test + public void testHsftpCustomDefaultPorts() throws IOException { + resetFileSystem(); + Configuration conf = new Configuration(); + conf.setInt("dfs.http.port", 123); + conf.setInt("dfs.https.port", 456); + + URI uri = URI.create("hsftp://localhost"); + HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf); + + assertEquals(456, fs.getDefaultPort()); + assertEquals(456, fs.getDefaultSecurePort()); + + assertEquals(uri, fs.getUri()); + assertEquals( + "127.0.0.1:456", + fs.getCanonicalServiceName() + ); + } + + @Test + public void testHsftpCustomUriPortWithDefaultPorts() throws IOException { + resetFileSystem(); + Configuration conf = new Configuration(); + URI uri = URI.create("hsftp://localhost:123"); + HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf); + + assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultPort()); + assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultSecurePort()); + + assertEquals(uri, fs.getUri()); + assertEquals( + "127.0.0.1:123", + fs.getCanonicalServiceName() + ); + } + + @Test + public void testHsftpCustomUriPortWithCustomDefaultPorts() throws IOException { + resetFileSystem(); + Configuration conf = new Configuration(); + conf.setInt("dfs.http.port", 123); + conf.setInt("dfs.https.port", 456); + + URI uri = URI.create("hsftp://localhost:789"); + HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf); + + assertEquals(456, fs.getDefaultPort()); + assertEquals(456, fs.getDefaultSecurePort()); + + assertEquals(uri, fs.getUri()); + assertEquals( + "127.0.0.1:789", + fs.getCanonicalServiceName() + ); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java index b810d1ba7f8..26acd0560ac 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java @@ -22,6 +22,8 @@ package org.apache.hadoop.hdfs.server.datanode; import java.io.File; import java.io.IOException; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; /** @@ -39,7 +41,18 @@ public class DataNodeTestUtils { return dn.getDNRegistrationForBP(bpid); } - public static File getBlockFile(FSDataset fsdataset, String bpid, long bid) { - return fsdataset.getFile(bpid, bid); + public static File getFile(DataNode dn, String bpid, long bid) { + return ((FSDataset)dn.getFSDataset()).getFile(bpid, bid); + } + + public static File getBlockFile(DataNode dn, String bpid, Block b + ) throws IOException { + return ((FSDataset)dn.getFSDataset()).getBlockFile(bpid, b); + } + + public static boolean unlinkBlock(DataNode dn, ExtendedBlock block, int numLinks + ) throws IOException { + ReplicaInfo info = ((FSDataset)dn.getFSDataset()).getReplicaInfo(block); + return info.unlinkBlock(numLinks); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java index 123810c9dc4..84ccb63423a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java @@ -116,10 +116,12 @@ public class TestEditLog extends TestCase { int numTransactions; short replication = 3; long blockSize = 64; + int startIndex; - Transactions(FSNamesystem ns, int num) { + Transactions(FSNamesystem ns, int numTx, int startIdx) { namesystem = ns; - numTransactions = num; + numTransactions = numTx; + startIndex = startIdx; } // add a bunch of transactions. @@ -131,8 +133,8 @@ public class TestEditLog extends TestCase { for (int i = 0; i < numTransactions; i++) { INodeFileUnderConstruction inode = new INodeFileUnderConstruction( p, replication, blockSize, 0, "", "", null); - editLog.logOpenFile("/filename" + i, inode); - editLog.logCloseFile("/filename" + i, inode); + editLog.logOpenFile("/filename" + startIndex + i, inode); + editLog.logCloseFile("/filename" + startIndex + i, inode); editLog.logSync(); } } @@ -280,7 +282,8 @@ public class TestEditLog extends TestCase { // Create threads and make them run transactions concurrently. Thread threadId[] = new Thread[NUM_THREADS]; for (int i = 0; i < NUM_THREADS; i++) { - Transactions trans = new Transactions(namesystem, NUM_TRANSACTIONS); + Transactions trans = + new Transactions(namesystem, NUM_TRANSACTIONS, i*NUM_TRANSACTIONS); threadId[i] = new Thread(trans, "TransactionThread-" + i); threadId[i].start(); } @@ -293,11 +296,16 @@ public class TestEditLog extends TestCase { i--; // retry } } - + + // Reopen some files as for append + Transactions trans = + new Transactions(namesystem, NUM_TRANSACTIONS, NUM_TRANSACTIONS / 2); + trans.run(); + // Roll another time to finalize edits_inprogress_3 fsimage.rollEditLog(); - long expectedTxns = (NUM_THREADS * 2 * NUM_TRANSACTIONS) + 2; // +2 for start/end txns + long expectedTxns = ((NUM_THREADS+1) * 2 * NUM_TRANSACTIONS) + 2; // +2 for start/end txns // Verify that we can read in all the transactions that we have written. // If there were any corruptions, it is likely that the reading in diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java index 83febfe79b7..17016640024 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java @@ -25,6 +25,7 @@ import javax.management.ObjectName; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.util.VersionInfo; import org.junit.Test; import junit.framework.Assert; @@ -57,6 +58,8 @@ public class TestNameNodeMXBean { // get attribute "Version" String version = (String) mbs.getAttribute(mxbeanName, "Version"); Assert.assertEquals(fsn.getVersion(), version); + Assert.assertTrue(version.equals(VersionInfo.getVersion() + + ", r" + VersionInfo.getRevision())); // get attribute "Used" Long used = (Long) mbs.getAttribute(mxbeanName, "Used"); Assert.assertEquals(fsn.getUsed(), used.longValue()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/ssl-client.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/ssl-client.xml new file mode 100644 index 00000000000..98910049ab3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/ssl-client.xml @@ -0,0 +1,26 @@ + + + + + + + + ssl.client.do.not.authenticate.server + true + + diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index b4bea69086b..8d39e40baf1 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -173,6 +173,13 @@ Release 0.23.1 - Unreleased MAPREDUCE-3774. Moved yarn-default.xml to hadoop-yarn-common from hadoop-server-common. (Mahadev Konar via vinodkv) + MAPREDUCE-3771. Un-deprecated the old mapred apis, port of MAPREDUCE-1735. + (acmurthy) + + MAPREDUCE-3784. Fixed CapacityScheduler so that maxActiveApplications and + maxActiveApplicationsPerUser per queue are not too low for small + clusters. (Arun C Murthy via vinodkv) + OPTIMIZATIONS MAPREDUCE-3567. Extraneous JobConf objects in AM heap. (Vinod Kumar @@ -621,6 +628,37 @@ Release 0.23.1 - Unreleased MAPREDUCE-3427. Fix streaming unit tests broken after mavenization. (Hitesh Shah via acmurthy) + MAPREDUCE-3640. Allow AMRecovery to work with partial JobHistory files. + (Arun C Murthy via sseth) + + MAPREDUCE-3752. Modified application limits to include queue max-capacities + besides the usual user limits. (Arun C Murthy via vinodkv) + + MAPREDUCE-3744. Fix the yarn logs command line. Improve error messages for + mapred job -logs. (Jason Lowe via sseth) + + MAPREDUCE-3780. Fixed a bug where applications killed before getting + activated were not getting cleaned up properly. (Hitesh Shah via acmurthy) + + MAPREDUCE-3708. Metrics: Incorrect Apps Submitted Count (Bhallamudi via + mahadev) + + MAPREDUCE-3727. jobtoken location property in jobconf refers to wrong + jobtoken file (tucu) + + MAPREDUCE-3711. Fixed MR AM recovery so that only single selected task + output is recovered and thus reduce the unnecessarily bloated recovery + time. (Robert Joseph Evans via vinodkv) + + MAPREDUCE-3760. Changed active nodes list to not contain unhealthy nodes + on the webUI and metrics. (vinodkv) + + MAPREDUCE-3417. Fixed job-access-controls to work with MR AM and + JobHistoryServer web-apps. (Jonathan Eagles via vinodkv) + + MAPREDUCE-3803. Fix broken build of raid contrib due to HDFS-2864. + (Ravi Prakash via suresh) + Release 0.23.0 - 2011-11-01 INCOMPATIBLE CHANGES diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java index a0e8613e206..c93696d4e05 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java @@ -431,9 +431,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job, @Override public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) { - if (!UserGroupInformation.isSecurityEnabled()) { - return true; - } AccessControlList jobACL = jobACLs.get(jobOperation); return aclsManager.checkAccess(callerUGI, jobOperation, username, jobACL); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java index fb2d32f4ab7..24a908112d2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java @@ -559,6 +559,7 @@ public abstract class TaskImpl implements Task, EventHandler { } private void internalError(TaskEventType type) { + LOG.error("Invalid event " + type + " on Task " + this.taskId); eventHandler.handle(new JobDiagnosticsUpdateEvent( this.taskId.getJobId(), "Invalid event " + type + " on Task " + this.taskId)); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java index f9e58b7ecc4..6bb51321430 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java @@ -103,6 +103,7 @@ public class LocalContainerAllocator extends RMCommunicator // This can happen when the connection to the RM has gone down. Keep // re-trying until the retryInterval has expired. if (System.currentTimeMillis() - retrystartTime >= retryInterval) { + LOG.error("Could not contact RM after " + retryInterval + " milliseconds."); eventHandler.handle(new JobEvent(this.getJob().getID(), JobEventType.INTERNAL_ERROR)); throw new YarnException("Could not contact RM after " + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/RecoveryService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/RecoveryService.java index 775cc11571e..e6831f83557 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/RecoveryService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/recover/RecoveryService.java @@ -32,8 +32,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.OutputCommitter; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; @@ -191,6 +193,11 @@ public class RecoveryService extends CompositeService implements Recovery { in = fc.open(historyFile); JobHistoryParser parser = new JobHistoryParser(in); jobInfo = parser.parse(); + Exception parseException = parser.getParseException(); + if (parseException != null) { + LOG.info("Got an error parsing job-history file " + historyFile + + ", ignoring incomplete events.", parseException); + } Map taskInfos = jobInfo .getAllTasks(); for (TaskInfo taskInfo : taskInfos.values()) { @@ -353,16 +360,24 @@ public class RecoveryService extends CompositeService implements Recovery { //recover the task output TaskAttemptContext taskContext = new TaskAttemptContextImpl(getConfig(), attInfo.getAttemptId()); - try { - committer.recoverTask(taskContext); + try { + TaskType type = taskContext.getTaskAttemptID().getTaskID().getTaskType(); + int numReducers = taskContext.getConfiguration().getInt(MRJobConfig.NUM_REDUCES, 1); + if(type == TaskType.REDUCE || (type == TaskType.MAP && numReducers <= 0)) { + committer.recoverTask(taskContext); + LOG.info("Recovered output from task attempt " + attInfo.getAttemptId()); + } else { + LOG.info("Will not try to recover output for " + + taskContext.getTaskAttemptID()); + } } catch (IOException e) { + LOG.error("Caught an exception while trying to recover task "+aId, e); actualHandler.handle(new JobDiagnosticsUpdateEvent( aId.getTaskId().getJobId(), "Error in recovering task output " + e.getMessage())); actualHandler.handle(new JobEvent(aId.getTaskId().getJobId(), JobEventType.INTERNAL_ERROR)); } - LOG.info("Recovered output from task attempt " + attInfo.getAttemptId()); // send the done event LOG.info("Sending done event to " + aId); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java index 4adbfc6d8d3..d188b7c42e9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java @@ -543,6 +543,7 @@ public class RMContainerAllocator extends RMContainerRequestor // This can happen when the connection to the RM has gone down. Keep // re-trying until the retryInterval has expired. if (System.currentTimeMillis() - retrystartTime >= retryInterval) { + LOG.error("Could not contact RM after " + retryInterval + " milliseconds."); eventHandler.handle(new JobEvent(this.getJob().getID(), JobEventType.INTERNAL_ERROR)); throw new YarnException("Could not contact RM after " + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java index 45ad63f0fb9..9b4b620de52 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java @@ -95,7 +95,13 @@ public class AppController extends Controller implements AMParams { * Render the /job page */ public void job() { - requireJob(); + try { + requireJob(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } render(jobPage()); } @@ -110,7 +116,13 @@ public class AppController extends Controller implements AMParams { * Render the /jobcounters page */ public void jobCounters() { - requireJob(); + try { + requireJob(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } if (app.getJob() != null) { setTitle(join("Counters for ", $(JOB_ID))); } @@ -121,7 +133,13 @@ public class AppController extends Controller implements AMParams { * Display a page showing a task's counters */ public void taskCounters() { - requireTask(); + try { + requireTask(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } if (app.getTask() != null) { setTitle(StringHelper.join("Counters for ", $(TASK_ID))); } @@ -140,7 +158,13 @@ public class AppController extends Controller implements AMParams { * @throws IOException on any error. */ public void singleJobCounter() throws IOException{ - requireJob(); + try { + requireJob(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } set(COUNTER_GROUP, URLDecoder.decode($(COUNTER_GROUP), "UTF-8")); set(COUNTER_NAME, URLDecoder.decode($(COUNTER_NAME), "UTF-8")); if (app.getJob() != null) { @@ -155,7 +179,13 @@ public class AppController extends Controller implements AMParams { * @throws IOException on any error. */ public void singleTaskCounter() throws IOException{ - requireTask(); + try { + requireTask(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } set(COUNTER_GROUP, URLDecoder.decode($(COUNTER_GROUP), "UTF-8")); set(COUNTER_NAME, URLDecoder.decode($(COUNTER_NAME), "UTF-8")); if (app.getTask() != null) { @@ -176,7 +206,13 @@ public class AppController extends Controller implements AMParams { * Render the /tasks page */ public void tasks() { - requireJob(); + try { + requireJob(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } if (app.getJob() != null) { try { String tt = $(TASK_TYPE); @@ -201,7 +237,13 @@ public class AppController extends Controller implements AMParams { * Render the /task page */ public void task() { - requireTask(); + try { + requireTask(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } if (app.getTask() != null) { setTitle(join("Attempts for ", $(TASK_ID))); } @@ -219,7 +261,13 @@ public class AppController extends Controller implements AMParams { * Render the attempts page */ public void attempts() { - requireJob(); + try { + requireJob(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } if (app.getJob() != null) { try { String taskType = $(TASK_TYPE); @@ -252,6 +300,13 @@ public class AppController extends Controller implements AMParams { */ public void conf() { requireJob(); + try { + requireJob(); + } + catch (Exception e) { + renderText(e.getMessage()); + return; + } render(confPage()); } @@ -280,41 +335,43 @@ public class AppController extends Controller implements AMParams { void accessDenied(String s) { setStatus(HttpServletResponse.SC_FORBIDDEN); setTitle(join("Access denied: ", s)); - throw new RuntimeException("Access denied: " + s); } /** * check for job access. * @param job the job that is being accessed + * @return True if the requesting user has permission to view the job */ - void checkAccess(Job job) { + boolean checkAccess(Job job) { UserGroupInformation callerUgi = UserGroupInformation.createRemoteUser( request().getRemoteUser()); - if (!job.checkAccess(callerUgi, JobACL.VIEW_JOB)) { - accessDenied("User " + request().getRemoteUser() + " does not have " + - " permissions."); - } + return job.checkAccess(callerUgi, JobACL.VIEW_JOB); } /** * Ensure that a JOB_ID was passed into the page. */ public void requireJob() { - try { - if ($(JOB_ID).isEmpty()) { - throw new RuntimeException("missing job ID"); - } - JobId jobID = MRApps.toJobID($(JOB_ID)); - app.setJob(app.context.getJob(jobID)); - if (app.getJob() == null) { - notFound($(JOB_ID)); - } - /* check for acl access */ - Job job = app.context.getJob(jobID); - checkAccess(job); - } catch (Exception e) { - badRequest(e.getMessage() == null ? - e.getClass().getName() : e.getMessage()); + if ($(JOB_ID).isEmpty()) { + badRequest("missing job ID"); + throw new RuntimeException("Bad Request: Missing job ID"); + } + + JobId jobID = MRApps.toJobID($(JOB_ID)); + app.setJob(app.context.getJob(jobID)); + if (app.getJob() == null) { + notFound($(JOB_ID)); + throw new RuntimeException("Not Found: " + $(JOB_ID)); + } + + /* check for acl access */ + Job job = app.context.getJob(jobID); + if (!checkAccess(job)) { + accessDenied("User " + request().getRemoteUser() + " does not have " + + " permission to view job " + $(JOB_ID)); + throw new RuntimeException("Access denied: User " + + request().getRemoteUser() + " does not have permission to view job " + + $(JOB_ID)); } } @@ -322,24 +379,30 @@ public class AppController extends Controller implements AMParams { * Ensure that a TASK_ID was passed into the page. */ public void requireTask() { - try { - if ($(TASK_ID).isEmpty()) { - throw new RuntimeException("missing task ID"); + if ($(TASK_ID).isEmpty()) { + badRequest("missing task ID"); + throw new RuntimeException("missing task ID"); + } + + TaskId taskID = MRApps.toTaskID($(TASK_ID)); + Job job = app.context.getJob(taskID.getJobId()); + app.setJob(job); + if (app.getJob() == null) { + notFound(MRApps.toString(taskID.getJobId())); + throw new RuntimeException("Not Found: " + $(JOB_ID)); + } else { + app.setTask(app.getJob().getTask(taskID)); + if (app.getTask() == null) { + notFound($(TASK_ID)); + throw new RuntimeException("Not Found: " + $(TASK_ID)); } - TaskId taskID = MRApps.toTaskID($(TASK_ID)); - Job job = app.context.getJob(taskID.getJobId()); - app.setJob(job); - if (app.getJob() == null) { - notFound(MRApps.toString(taskID.getJobId())); - } else { - app.setTask(app.getJob().getTask(taskID)); - if (app.getTask() == null) { - notFound($(TASK_ID)); - } - } - checkAccess(job); - } catch (Exception e) { - badRequest(e.getMessage()); + } + if (!checkAccess(job)) { + accessDenied("User " + request().getRemoteUser() + " does not have " + + " permission to view job " + $(JOB_ID)); + throw new RuntimeException("Access denied: User " + + request().getRemoteUser() + " does not have permission to view job " + + $(JOB_ID)); } } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java index ec492de7fed..376227b51ef 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java @@ -61,6 +61,7 @@ import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.junit.Test; +@SuppressWarnings({"unchecked", "rawtypes"}) public class TestRecovery { private static final Log LOG = LogFactory.getLog(TestRecovery.class); @@ -112,7 +113,7 @@ public class TestRecovery { Assert.assertEquals("Reduce Task state not correct", TaskState.RUNNING, reduceTask.getReport().getTaskState()); - //send the fail signal to the 1st map task attempt + //send the fail signal to the 1st map task attempt app.getContext().getEventHandler().handle( new TaskAttemptEvent( task1Attempt1.getID(), @@ -193,7 +194,7 @@ public class TestRecovery { //RUNNING state app.waitForState(task2Attempt, TaskAttemptState.RUNNING); - //send the done signal to the 2nd map task + //send the done signal to the 2nd map task app.getContext().getEventHandler().handle( new TaskAttemptEvent( mapTask2.getAttempts().values().iterator().next().getID(), @@ -349,6 +350,151 @@ public class TestRecovery { validateOutput(); } + @Test + public void testOutputRecoveryMapsOnly() throws Exception { + int runCount = 0; + MRApp app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), + true, ++runCount); + Configuration conf = new Configuration(); + conf.setBoolean("mapred.mapper.new-api", true); + conf.setBoolean("mapred.reducer.new-api", true); + conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); + conf.set(FileOutputFormat.OUTDIR, outputDir.toString()); + Job job = app.submit(conf); + app.waitForState(job, JobState.RUNNING); + Assert.assertEquals("No of tasks not correct", + 3, job.getTasks().size()); + Iterator it = job.getTasks().values().iterator(); + Task mapTask1 = it.next(); + Task mapTask2 = it.next(); + Task reduceTask1 = it.next(); + + // all maps must be running + app.waitForState(mapTask1, TaskState.RUNNING); + + TaskAttempt task1Attempt1 = mapTask1.getAttempts().values().iterator() + .next(); + + //before sending the TA_DONE, event make sure attempt has come to + //RUNNING state + app.waitForState(task1Attempt1, TaskAttemptState.RUNNING); + + // write output corresponding to map1 (This is just to validate that it is + //no included in the output) + writeBadOutput(task1Attempt1, conf); + + //send the done signal to the map + app.getContext().getEventHandler().handle( + new TaskAttemptEvent( + task1Attempt1.getID(), + TaskAttemptEventType.TA_DONE)); + + //wait for map task to complete + app.waitForState(mapTask1, TaskState.SUCCEEDED); + + // Verify the shuffle-port + Assert.assertEquals(5467, task1Attempt1.getShufflePort()); + + //stop the app before the job completes. + app.stop(); + + //rerun + //in rerun the map will be recovered from previous run + app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), false, + ++runCount); + conf = new Configuration(); + conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true); + conf.setBoolean("mapred.mapper.new-api", true); + conf.setBoolean("mapred.reducer.new-api", true); + conf.set(FileOutputFormat.OUTDIR, outputDir.toString()); + conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); + job = app.submit(conf); + app.waitForState(job, JobState.RUNNING); + Assert.assertEquals("No of tasks not correct", + 3, job.getTasks().size()); + it = job.getTasks().values().iterator(); + mapTask1 = it.next(); + mapTask2 = it.next(); + reduceTask1 = it.next(); + + // map will be recovered, no need to send done + app.waitForState(mapTask1, TaskState.SUCCEEDED); + + // Verify the shuffle-port after recovery + task1Attempt1 = mapTask1.getAttempts().values().iterator().next(); + Assert.assertEquals(5467, task1Attempt1.getShufflePort()); + + app.waitForState(mapTask2, TaskState.RUNNING); + + TaskAttempt task2Attempt1 = mapTask2.getAttempts().values().iterator() + .next(); + + //before sending the TA_DONE, event make sure attempt has come to + //RUNNING state + app.waitForState(task2Attempt1, TaskAttemptState.RUNNING); + + //send the done signal to the map + app.getContext().getEventHandler().handle( + new TaskAttemptEvent( + task2Attempt1.getID(), + TaskAttemptEventType.TA_DONE)); + + //wait for map task to complete + app.waitForState(mapTask2, TaskState.SUCCEEDED); + + // Verify the shuffle-port + Assert.assertEquals(5467, task2Attempt1.getShufflePort()); + + app.waitForState(reduceTask1, TaskState.RUNNING); + TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next(); + + // write output corresponding to reduce1 + writeOutput(reduce1Attempt1, conf); + + //send the done signal to the 1st reduce + app.getContext().getEventHandler().handle( + new TaskAttemptEvent( + reduce1Attempt1.getID(), + TaskAttemptEventType.TA_DONE)); + + //wait for first reduce task to complete + app.waitForState(reduceTask1, TaskState.SUCCEEDED); + + app.waitForState(job, JobState.SUCCEEDED); + app.verifyCompleted(); + validateOutput(); + } + + private void writeBadOutput(TaskAttempt attempt, Configuration conf) + throws Exception { + TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, + TypeConverter.fromYarn(attempt.getID())); + + TextOutputFormat theOutputFormat = new TextOutputFormat(); + RecordWriter theRecordWriter = theOutputFormat + .getRecordWriter(tContext); + + NullWritable nullWritable = NullWritable.get(); + try { + theRecordWriter.write(key2, val2); + theRecordWriter.write(null, nullWritable); + theRecordWriter.write(null, val2); + theRecordWriter.write(nullWritable, val1); + theRecordWriter.write(key1, nullWritable); + theRecordWriter.write(key2, null); + theRecordWriter.write(null, null); + theRecordWriter.write(key1, val1); + } finally { + theRecordWriter.close(tContext); + } + + OutputFormat outputFormat = ReflectionUtils.newInstance( + tContext.getOutputFormatClass(), conf); + OutputCommitter committer = outputFormat.getOutputCommitter(tContext); + committer.commitTask(tContext); +} + + private void writeOutput(TaskAttempt attempt, Configuration conf) throws Exception { TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java index a142c085fa6..acec4cb6c3d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java @@ -28,6 +28,11 @@ import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.OutputCommitter; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; +import org.apache.hadoop.mapreduce.JobACL; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.MRConfig; +import org.apache.hadoop.mapreduce.TypeConverter; +import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent; import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl; import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.JobNoTasksCompletedTransition; @@ -37,6 +42,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobState; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics; import org.apache.hadoop.mapreduce.v2.app.MRApp; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.event.EventHandler; import org.junit.Test; import org.junit.Assert; @@ -134,4 +140,61 @@ public class TestJobImpl { t.testCheckJobCompleteSuccess(); t.testCheckJobCompleteSuccessFailed(); } + + @Test + public void testCheckAccess() { + // Create two unique users + String user1 = System.getProperty("user.name"); + String user2 = user1 + "1234"; + UserGroupInformation ugi1 = UserGroupInformation.createRemoteUser(user1); + UserGroupInformation ugi2 = UserGroupInformation.createRemoteUser(user2); + + // Create the job + JobID jobID = JobID.forName("job_1234567890000_0001"); + JobId jobId = TypeConverter.toYarn(jobID); + + // Setup configuration access only to user1 (owner) + Configuration conf1 = new Configuration(); + conf1.setBoolean(MRConfig.MR_ACLS_ENABLED, true); + conf1.set(MRJobConfig.JOB_ACL_VIEW_JOB, ""); + + // Verify access + JobImpl job1 = new JobImpl(jobId, null, conf1, null, null, null, null, null, + null, null, null, true, null, 0, null); + Assert.assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB)); + Assert.assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB)); + + // Setup configuration access to the user1 (owner) and user2 + Configuration conf2 = new Configuration(); + conf2.setBoolean(MRConfig.MR_ACLS_ENABLED, true); + conf2.set(MRJobConfig.JOB_ACL_VIEW_JOB, user2); + + // Verify access + JobImpl job2 = new JobImpl(jobId, null, conf2, null, null, null, null, null, + null, null, null, true, null, 0, null); + Assert.assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB)); + Assert.assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB)); + + // Setup configuration access with security enabled and access to all + Configuration conf3 = new Configuration(); + conf3.setBoolean(MRConfig.MR_ACLS_ENABLED, true); + conf3.set(MRJobConfig.JOB_ACL_VIEW_JOB, "*"); + + // Verify access + JobImpl job3 = new JobImpl(jobId, null, conf3, null, null, null, null, null, + null, null, null, true, null, 0, null); + Assert.assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB)); + Assert.assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB)); + + // Setup configuration access without security enabled + Configuration conf4 = new Configuration(); + conf4.setBoolean(MRConfig.MR_ACLS_ENABLED, false); + conf4.set(MRJobConfig.JOB_ACL_VIEW_JOB, ""); + + // Verify access + JobImpl job4 = new JobImpl(jobId, null, conf4, null, null, null, null, null, + null, null, null, true, null, 0, null); + Assert.assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB)); + Assert.assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB)); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/filecache/DistributedCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/filecache/DistributedCache.java index fc623210d81..3b9424a81c9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/filecache/DistributedCache.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/filecache/DistributedCache.java @@ -119,9 +119,7 @@ import org.apache.hadoop.mapreduce.Job; * @see org.apache.hadoop.mapred.JobConf * @see org.apache.hadoop.mapred.JobClient * @see org.apache.hadoop.mapreduce.Job - * @deprecated Use methods on {@link Job}. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class DistributedCache extends diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/filecache/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/filecache/package-info.java index 499007c16aa..7e0f20d1bf1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/filecache/package-info.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/filecache/package-info.java @@ -16,8 +16,4 @@ * limitations under the License. * */ -/** - * Deprecated. Use {@link org.apache.hadoop.mapreduce.Job} instead. - */ -@Deprecated package org.apache.hadoop.filecache; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java index 7c8f8eb5f47..00775d2c2eb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java @@ -62,9 +62,7 @@ import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus; * {@link JobClient#getClusterStatus()}.

* * @see JobClient - * @deprecated Use {@link ClusterMetrics} or {@link TaskTrackerInfo} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class ClusterStatus implements Writable { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Counters.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Counters.java index 9d8a2a71da8..b739402ffb6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Counters.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Counters.java @@ -18,27 +18,24 @@ package org.apache.hadoop.mapred; +import static org.apache.hadoop.mapreduce.util.CountersStrings.parseEscapedCompactString; +import static org.apache.hadoop.mapreduce.util.CountersStrings.toEscapedCompactString; + import java.text.ParseException; import org.apache.commons.logging.Log; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableUtils; -import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter; -import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.mapreduce.FileSystemCounter; import org.apache.hadoop.mapreduce.counters.AbstractCounterGroup; import org.apache.hadoop.mapreduce.counters.AbstractCounters; import org.apache.hadoop.mapreduce.counters.CounterGroupBase; import org.apache.hadoop.mapreduce.counters.CounterGroupFactory; -import org.apache.hadoop.mapreduce.counters.FrameworkCounterGroup; import org.apache.hadoop.mapreduce.counters.FileSystemCounterGroup; +import org.apache.hadoop.mapreduce.counters.FrameworkCounterGroup; import org.apache.hadoop.mapreduce.counters.GenericCounter; import org.apache.hadoop.mapreduce.counters.Limits; -import static org.apache.hadoop.mapreduce.util.CountersStrings.*; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter; /** * A set of named counters. @@ -49,9 +46,7 @@ import static org.apache.hadoop.mapreduce.util.CountersStrings.*; * *

Counters are bunched into {@link Group}s, each comprising of * counters from a particular Enum class. - * @deprecated Use {@link org.apache.hadoop.mapreduce.Counters} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class Counters diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileAlreadyExistsException.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileAlreadyExistsException.java index 19c3f4c6223..85017174ef4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileAlreadyExistsException.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileAlreadyExistsException.java @@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability; * Used when target file already exists for any operation and * is not configured to be overwritten. */ -@Deprecated // may be removed after 0.23 @InterfaceAudience.Public @InterfaceStability.Stable public class FileAlreadyExistsException diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java index aaf3c26b789..4f7e4b95fd3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java @@ -54,10 +54,7 @@ import org.apache.hadoop.util.StringUtils; * Subclasses of FileInputFormat can also override the * {@link #isSplitable(FileSystem, Path)} method to ensure input-files are * not split-up and are processed as a whole by {@link Mapper}s. - * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat} - * instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class FileInputFormat implements InputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java index 32b6e2232d0..c23e9a93789 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java @@ -19,14 +19,12 @@ package org.apache.hadoop.mapred; import java.io.IOException; -import java.net.URI; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; /** An {@link OutputCommitter} that commits files specified @@ -42,280 +40,140 @@ public class FileOutputCommitter extends OutputCommitter { /** * Temporary directory name */ - public static final String TEMP_DIR_NAME = "_temporary"; - public static final String SUCCEEDED_FILE_NAME = "_SUCCESS"; - static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = - "mapreduce.fileoutputcommitter.marksuccessfuljobs"; + public static final String TEMP_DIR_NAME = + org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.PENDING_DIR_NAME; + public static final String SUCCEEDED_FILE_NAME = + org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCEEDED_FILE_NAME; + static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = + org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER; + + private static Path getOutputPath(JobContext context) { + JobConf conf = context.getJobConf(); + return FileOutputFormat.getOutputPath(conf); + } + + private static Path getOutputPath(TaskAttemptContext context) { + JobConf conf = context.getJobConf(); + return FileOutputFormat.getOutputPath(conf); + } + + private org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter wrapped = null; + + private org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + getWrapped(JobContext context) throws IOException { + if(wrapped == null) { + wrapped = new org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter( + getOutputPath(context), context); + } + return wrapped; + } + + private org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + getWrapped(TaskAttemptContext context) throws IOException { + if(wrapped == null) { + wrapped = new org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter( + getOutputPath(context), context); + } + return wrapped; + } + + /** + * Compute the path where the output of a given job attempt will be placed. + * @param context the context of the job. This is used to get the + * application attempt id. + * @return the path to store job attempt data. + */ + @Private + Path getJobAttemptPath(JobContext context) { + return org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + .getJobAttemptPath(context, getOutputPath(context)); + } + @Private + Path getTaskAttemptPath(TaskAttemptContext context) throws IOException { + return getTaskAttemptPath(context, getOutputPath(context)); + } + + private Path getTaskAttemptPath(TaskAttemptContext context, Path out) throws IOException { + Path workPath = FileOutputFormat.getWorkOutputPath(context.getJobConf()); + if(workPath == null) { + return org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + .getTaskAttemptPath(context, out); + } + return workPath; + } + + /** + * Compute the path where the output of a committed task is stored until + * the entire job is committed. + * @param context the context of the task attempt + * @return the path where the output of a committed task is stored until + * the entire job is committed. + */ + Path getCommittedTaskPath(TaskAttemptContext context) { + return org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + .getCommittedTaskPath(context, getOutputPath(context)); + } + + public Path getWorkPath(TaskAttemptContext context, Path outputPath) + throws IOException { + return getTaskAttemptPath(context, outputPath); + } + + @Override public void setupJob(JobContext context) throws IOException { - JobConf conf = context.getJobConf(); - Path outputPath = FileOutputFormat.getOutputPath(conf); - if (outputPath != null) { - Path tmpDir = - new Path(outputPath, getJobAttemptBaseDirName(context) + - Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME); - FileSystem fileSys = tmpDir.getFileSystem(conf); - if (!fileSys.mkdirs(tmpDir)) { - LOG.error("Mkdirs failed to create " + tmpDir.toString()); - } - } - } - - // True if the job requires output.dir marked on successful job. - // Note that by default it is set to true. - private boolean shouldMarkOutputDir(JobConf conf) { - return conf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true); + getWrapped(context).setupJob(context); } + @Override public void commitJob(JobContext context) throws IOException { - //delete the task temp directory from the current jobtempdir - JobConf conf = context.getJobConf(); - Path outputPath = FileOutputFormat.getOutputPath(conf); - if (outputPath != null) { - FileSystem outputFileSystem = outputPath.getFileSystem(conf); - Path tmpDir = new Path(outputPath, getJobAttemptBaseDirName(context) + - Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME); - FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration()); - if (fileSys.exists(tmpDir)) { - fileSys.delete(tmpDir, true); - } else { - LOG.warn("Task temp dir could not be deleted " + tmpDir); - } - - //move the job output to final place - Path jobOutputPath = - new Path(outputPath, getJobAttemptBaseDirName(context)); - moveJobOutputs(outputFileSystem, - jobOutputPath, outputPath, jobOutputPath); - - // delete the _temporary folder in the output folder - cleanupJob(context); - // check if the output-dir marking is required - if (shouldMarkOutputDir(context.getJobConf())) { - // create a _success file in the output folder - markOutputDirSuccessful(context); - } - } - } - - // Create a _success file in the job's output folder - private void markOutputDirSuccessful(JobContext context) throws IOException { - JobConf conf = context.getJobConf(); - // get the o/p path - Path outputPath = FileOutputFormat.getOutputPath(conf); - if (outputPath != null) { - // get the filesys - FileSystem fileSys = outputPath.getFileSystem(conf); - // create a file in the output folder to mark the job completion - Path filePath = new Path(outputPath, SUCCEEDED_FILE_NAME); - fileSys.create(filePath).close(); - } - } - - private void moveJobOutputs(FileSystem fs, final Path origJobOutputPath, - Path finalOutputDir, Path jobOutput) throws IOException { - LOG.debug("Told to move job output from " + jobOutput - + " to " + finalOutputDir + - " and orig job output path is " + origJobOutputPath); - if (fs.isFile(jobOutput)) { - Path finalOutputPath = - getFinalPath(fs, finalOutputDir, jobOutput, origJobOutputPath); - if (!fs.rename(jobOutput, finalOutputPath)) { - if (!fs.delete(finalOutputPath, true)) { - throw new IOException("Failed to delete earlier output of job"); - } - if (!fs.rename(jobOutput, finalOutputPath)) { - throw new IOException("Failed to save output of job"); - } - } - LOG.debug("Moved job output file from " + jobOutput + " to " + - finalOutputPath); - } else if (fs.getFileStatus(jobOutput).isDirectory()) { - LOG.debug("Job output file " + jobOutput + " is a dir"); - FileStatus[] paths = fs.listStatus(jobOutput); - Path finalOutputPath = - getFinalPath(fs, finalOutputDir, jobOutput, origJobOutputPath); - fs.mkdirs(finalOutputPath); - LOG.debug("Creating dirs along job output path " + finalOutputPath); - if (paths != null) { - for (FileStatus path : paths) { - moveJobOutputs(fs, origJobOutputPath, finalOutputDir, path.getPath()); - } - } - } + getWrapped(context).commitJob(context); } @Override @Deprecated public void cleanupJob(JobContext context) throws IOException { - JobConf conf = context.getJobConf(); - // do the clean up of temporary directory - Path outputPath = FileOutputFormat.getOutputPath(conf); - if (outputPath != null) { - Path tmpDir = new Path(outputPath, FileOutputCommitter.TEMP_DIR_NAME); - FileSystem fileSys = tmpDir.getFileSystem(conf); - context.getProgressible().progress(); - if (fileSys.exists(tmpDir)) { - fileSys.delete(tmpDir, true); - } else { - LOG.warn("Output Path is Null in cleanup"); - } - } + getWrapped(context).cleanupJob(context); } @Override public void abortJob(JobContext context, int runState) throws IOException { - // simply delete the _temporary dir from the o/p folder of the job - cleanupJob(context); + JobStatus.State state; + if(runState == JobStatus.State.RUNNING.getValue()) { + state = JobStatus.State.RUNNING; + } else if(runState == JobStatus.State.SUCCEEDED.getValue()) { + state = JobStatus.State.SUCCEEDED; + } else if(runState == JobStatus.State.FAILED.getValue()) { + state = JobStatus.State.FAILED; + } else if(runState == JobStatus.State.PREP.getValue()) { + state = JobStatus.State.PREP; + } else if(runState == JobStatus.State.KILLED.getValue()) { + state = JobStatus.State.KILLED; + } else { + throw new IllegalArgumentException(runState+" is not a valid runState."); + } + getWrapped(context).abortJob(context, state); } public void setupTask(TaskAttemptContext context) throws IOException { - // FileOutputCommitter's setupTask doesn't do anything. Because the - // temporary task directory is created on demand when the - // task is writing. - } - - public void commitTask(TaskAttemptContext context) - throws IOException { - Path taskOutputPath = getTempTaskOutputPath(context); - TaskAttemptID attemptId = context.getTaskAttemptID(); - JobConf job = context.getJobConf(); - if (taskOutputPath != null) { - FileSystem fs = taskOutputPath.getFileSystem(job); - context.getProgressible().progress(); - if (fs.exists(taskOutputPath)) { - // Move the task outputs to the current job attempt output dir - JobConf conf = context.getJobConf(); - Path outputPath = FileOutputFormat.getOutputPath(conf); - FileSystem outputFileSystem = outputPath.getFileSystem(conf); - Path jobOutputPath = new Path(outputPath, getJobTempDirName(context)); - moveTaskOutputs(context, outputFileSystem, jobOutputPath, - taskOutputPath); - - // Delete the temporary task-specific output directory - if (!fs.delete(taskOutputPath, true)) { - LOG.info("Failed to delete the temporary output" + - " directory of task: " + attemptId + " - " + taskOutputPath); - } - LOG.info("Saved output of task '" + attemptId + "' to " + - jobOutputPath); - } - } - } - - private void moveTaskOutputs(TaskAttemptContext context, - FileSystem fs, - Path jobOutputDir, - Path taskOutput) - throws IOException { - TaskAttemptID attemptId = context.getTaskAttemptID(); - context.getProgressible().progress(); - LOG.debug("Told to move taskoutput from " + taskOutput - + " to " + jobOutputDir); - if (fs.isFile(taskOutput)) { - Path finalOutputPath = getFinalPath(fs, jobOutputDir, taskOutput, - getTempTaskOutputPath(context)); - if (!fs.rename(taskOutput, finalOutputPath)) { - if (!fs.delete(finalOutputPath, true)) { - throw new IOException("Failed to delete earlier output of task: " + - attemptId); - } - if (!fs.rename(taskOutput, finalOutputPath)) { - throw new IOException("Failed to save output of task: " + - attemptId); - } - } - LOG.debug("Moved " + taskOutput + " to " + finalOutputPath); - } else if(fs.getFileStatus(taskOutput).isDirectory()) { - LOG.debug("Taskoutput " + taskOutput + " is a dir"); - FileStatus[] paths = fs.listStatus(taskOutput); - Path finalOutputPath = getFinalPath(fs, jobOutputDir, taskOutput, - getTempTaskOutputPath(context)); - fs.mkdirs(finalOutputPath); - LOG.debug("Creating dirs along path " + finalOutputPath); - if (paths != null) { - for (FileStatus path : paths) { - moveTaskOutputs(context, fs, jobOutputDir, path.getPath()); - } - } - } - } - - public void abortTask(TaskAttemptContext context) throws IOException { - Path taskOutputPath = getTempTaskOutputPath(context); - if (taskOutputPath != null) { - FileSystem fs = taskOutputPath.getFileSystem(context.getJobConf()); - context.getProgressible().progress(); - fs.delete(taskOutputPath, true); - } - } - - @SuppressWarnings("deprecation") - private Path getFinalPath(FileSystem fs, Path jobOutputDir, Path taskOutput, - Path taskOutputPath) throws IOException { - URI taskOutputUri = taskOutput.makeQualified(fs).toUri(); - URI taskOutputPathUri = taskOutputPath.makeQualified(fs).toUri(); - URI relativePath = taskOutputPathUri.relativize(taskOutputUri); - if (taskOutputUri == relativePath) { - //taskOutputPath is not a parent of taskOutput - throw new IOException("Can not get the relative path: base = " + - taskOutputPathUri + " child = " + taskOutputUri); - } - if (relativePath.getPath().length() > 0) { - return new Path(jobOutputDir, relativePath.getPath()); - } else { - return jobOutputDir; - } - } - - public boolean needsTaskCommit(TaskAttemptContext context) - throws IOException { - Path taskOutputPath = getTempTaskOutputPath(context); - if (taskOutputPath != null) { - context.getProgressible().progress(); - // Get the file-system for the task output directory - FileSystem fs = taskOutputPath.getFileSystem(context.getJobConf()); - // since task output path is created on demand, - // if it exists, task needs a commit - if (fs.exists(taskOutputPath)) { - return true; - } - } - return false; - } - - Path getTempTaskOutputPath(TaskAttemptContext taskContext) - throws IOException { - JobConf conf = taskContext.getJobConf(); - Path outputPath = FileOutputFormat.getOutputPath(conf); - if (outputPath != null) { - Path p = new Path(outputPath, - (FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + - "_" + taskContext.getTaskAttemptID().toString())); - FileSystem fs = p.getFileSystem(conf); - return p.makeQualified(fs); - } - return null; + getWrapped(context).setupTask(context); } - Path getWorkPath(TaskAttemptContext taskContext, Path basePath) + @Override + public void commitTask(TaskAttemptContext context) throws IOException { + getWrapped(context).commitTask(context, getTaskAttemptPath(context)); + } + + @Override + public void abortTask(TaskAttemptContext context) throws IOException { + getWrapped(context).abortTask(context, getTaskAttemptPath(context)); + } + + @Override + public boolean needsTaskCommit(TaskAttemptContext context) throws IOException { - // ${mapred.out.dir}/_temporary - Path jobTmpDir = new Path(basePath, FileOutputCommitter.TEMP_DIR_NAME); - FileSystem fs = jobTmpDir.getFileSystem(taskContext.getJobConf()); - if (!fs.exists(jobTmpDir)) { - throw new IOException("The temporary job-output directory " + - jobTmpDir.toString() + " doesn't exist!"); - } - // ${mapred.out.dir}/_temporary/_${taskid} - String taskid = taskContext.getTaskAttemptID().toString(); - Path taskTmpDir = new Path(jobTmpDir, "_" + taskid); - if (!fs.mkdirs(taskTmpDir)) { - throw new IOException("Mkdirs failed to create " - + taskTmpDir.toString()); - } - return taskTmpDir; + return getWrapped(context).needsTaskCommit(context, getTaskAttemptPath(context)); } @Override @@ -326,54 +184,6 @@ public class FileOutputCommitter extends OutputCommitter { @Override public void recoverTask(TaskAttemptContext context) throws IOException { - Path outputPath = FileOutputFormat.getOutputPath(context.getJobConf()); - context.progress(); - Path jobOutputPath = new Path(outputPath, getJobTempDirName(context)); - int previousAttempt = - context.getConfiguration().getInt( - MRConstants.APPLICATION_ATTEMPT_ID, 0) - 1; - if (previousAttempt < 0) { - LOG.warn("Cannot recover task output for first attempt..."); - return; - } - - FileSystem outputFileSystem = - outputPath.getFileSystem(context.getJobConf()); - Path pathToRecover = - new Path(outputPath, getJobAttemptBaseDirName(previousAttempt)); - if (outputFileSystem.exists(pathToRecover)) { - // Move the task outputs to their final place - LOG.debug("Trying to recover task from " + pathToRecover - + " into " + jobOutputPath); - moveJobOutputs(outputFileSystem, - pathToRecover, jobOutputPath, pathToRecover); - LOG.info("Saved output of job to " + jobOutputPath); - } - } - - protected static String getJobAttemptBaseDirName(JobContext context) { - int appAttemptId = - context.getJobConf().getInt( - MRConstants.APPLICATION_ATTEMPT_ID, 0); - return getJobAttemptBaseDirName(appAttemptId); - } - - protected static String getJobTempDirName(TaskAttemptContext context) { - int appAttemptId = - context.getJobConf().getInt( - MRConstants.APPLICATION_ATTEMPT_ID, 0); - return getJobAttemptBaseDirName(appAttemptId); - } - - protected static String getJobAttemptBaseDirName(int appAttemptId) { - return FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + - + appAttemptId; - } - - protected static String getTaskAttemptBaseDirName( - TaskAttemptContext context) { - return getJobTempDirName(context) + Path.SEPARATOR + - FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + - "_" + context.getTaskAttemptID().toString(); + getWrapped(context).recoverTask(context); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileSplit.java index 7366e9055c9..fb1c651a9a9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileSplit.java @@ -29,10 +29,7 @@ import org.apache.hadoop.fs.Path; /** A section of an input file. Returned by {@link * InputFormat#getSplits(JobConf, int)} and passed to * {@link InputFormat#getRecordReader(InputSplit,JobConf,Reporter)}. - * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.input.FileSplit} - * instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class FileSplit extends org.apache.hadoop.mapreduce.InputSplit diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ID.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ID.java index 833f8fb5657..abcd70784e8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ID.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ID.java @@ -30,7 +30,6 @@ import org.apache.hadoop.classification.InterfaceStability; * @see TaskID * @see TaskAttemptID */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class ID extends org.apache.hadoop.mapreduce.ID { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputFormat.java index 9743ca38bad..8179c9a5120 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputFormat.java @@ -63,9 +63,7 @@ import org.apache.hadoop.fs.FileSystem; * @see RecordReader * @see JobClient * @see FileInputFormat - * @deprecated Use {@link org.apache.hadoop.mapreduce.InputFormat} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface InputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputSplit.java index 4951b8f386e..593f4721d35 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputSplit.java @@ -34,9 +34,7 @@ import org.apache.hadoop.io.Writable; * * @see InputFormat * @see RecordReader - * @deprecated Use {@link org.apache.hadoop.mapreduce.InputSplit} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface InputSplit extends Writable { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java index 7cce583c674..1e3f952303e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java @@ -29,6 +29,9 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.ClusterStatus.BlackListInfo; import org.apache.hadoop.mapreduce.Cluster; import org.apache.hadoop.mapreduce.ClusterMetrics; @@ -40,13 +43,10 @@ import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.mapreduce.tools.CLI; import org.apache.hadoop.mapreduce.util.ConfigUtil; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenRenewer; -import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -132,9 +132,7 @@ import org.apache.hadoop.util.ToolRunner; * @see ClusterStatus * @see Tool * @see DistributedCache - * @deprecated Use {@link Job} and {@link Cluster} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class JobClient extends CLI { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java index 0ff0a43284a..4d6787918f9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java @@ -20,7 +20,6 @@ package org.apache.hadoop.mapred; import java.io.IOException; - import java.net.URL; import java.net.URLDecoder; import java.util.Enumeration; @@ -28,24 +27,26 @@ import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.mapreduce.filecache.DistributedCache; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; - -import org.apache.hadoop.io.*; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.io.compress.CompressionCodec; - +import org.apache.hadoop.mapred.lib.HashPartitioner; import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.IdentityReducer; -import org.apache.hadoop.mapred.lib.HashPartitioner; import org.apache.hadoop.mapred.lib.KeyFieldBasedComparator; import org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.mapreduce.util.ConfigUtil; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.util.ReflectionUtils; @@ -107,9 +108,7 @@ import org.apache.log4j.Level; * @see ClusterStatus * @see Tool * @see DistributedCache - * @deprecated Use {@link Configuration} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class JobConf extends Configuration { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConfigurable.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConfigurable.java index 3e75acd495a..9dffce828b5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConfigurable.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConfigurable.java @@ -22,7 +22,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; /** That what may be configured. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface JobConfigurable { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContext.java index 62d379b70d2..83a3c094115 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContext.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContext.java @@ -22,10 +22,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.util.Progressable; -/** - * @deprecated Use {@link org.apache.hadoop.mapreduce.JobContext} instead. - */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface JobContext extends org.apache.hadoop.mapreduce.JobContext { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContextImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContextImpl.java index 8e188903b7d..faf9e286cd3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContextImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContextImpl.java @@ -21,10 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.util.Progressable; -/** - * @deprecated Use {@link org.apache.hadoop.mapreduce.JobContext} instead. - */ -@Deprecated @InterfaceAudience.Private @InterfaceStability.Unstable public class JobContextImpl diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobID.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobID.java index 63dd2abcb46..699a939d908 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobID.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobID.java @@ -41,7 +41,6 @@ import org.apache.hadoop.classification.InterfaceStability; * @see TaskID * @see TaskAttemptID */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class JobID extends org.apache.hadoop.mapreduce.JobID { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobPriority.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobPriority.java index 861a05ff5f9..376d8a410f0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobPriority.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobPriority.java @@ -22,9 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * Used to describe the priority of the running job. - * @deprecated Use {@link org.apache.hadoop.mapreduce.JobPriority} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public enum JobPriority { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java index 41e6d09ebdd..3bfc7722deb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java @@ -29,9 +29,7 @@ import org.apache.hadoop.mapreduce.QueueState; /** * Class that contains the information regarding the Job Queues which are * maintained by the Hadoop Map/Reduce framework. - * @deprecated Use {@link QueueInfo} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class JobQueueInfo extends QueueInfo { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java index c10a4c0a640..2b3c9509cf2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java @@ -29,9 +29,7 @@ import org.apache.hadoop.security.authorize.AccessControlList; * not intended to be a comprehensive piece of data. * For that, look at JobProfile. ************************************************* - *@deprecated Use {@link org.apache.hadoop.mapreduce.JobStatus} instead **/ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueLineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueLineRecordReader.java index 09ced3bba71..2c3e83cb958 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueLineRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueLineRecordReader.java @@ -31,12 +31,7 @@ import org.apache.hadoop.io.Text; * separator character. The separator can be specified in config file * under the attribute name mapreduce.input.keyvaluelinerecordreader.key.value.separator. The default * separator is the tab character ('\t'). - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.KeyValueLineRecordReader} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class KeyValueLineRecordReader implements RecordReader { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueTextInputFormat.java index d60e028f7f8..f8097f49d67 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueTextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueTextInputFormat.java @@ -34,12 +34,7 @@ import org.apache.hadoop.io.compress.SplittableCompressionCodec; * Either linefeed or carriage-return are used to signal end of line. Each line * is divided into key and value parts by a separator byte. If no such a byte * exists, the key will be the entire line and value will be empty. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.KeyValueTextInputFormat} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class KeyValueTextInputFormat extends FileInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java index 0da0f75090d..35755da9913 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java @@ -41,10 +41,7 @@ import org.apache.commons.logging.Log; /** * Treats keys as offset in file and value as line. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.LineRecordReader} instead. */ -@Deprecated @InterfaceAudience.LimitedPrivate({"MapReduce", "Pig"}) @InterfaceStability.Unstable public class LineRecordReader implements RecordReader { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java index a9e25f287d4..cb480a8f1cb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java @@ -23,7 +23,6 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.MRConfig; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapFileOutputFormat.java index 3093c8a6ce7..374a6c79ae9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapFileOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapFileOutputFormat.java @@ -36,10 +36,7 @@ import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.ReflectionUtils; /** An {@link OutputFormat} that writes {@link MapFile}s. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.MapFileOutputFormat} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class MapFileOutputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapReduceBase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapReduceBase.java index 0281cf3eb15..f2f543f40a1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapReduceBase.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapReduceBase.java @@ -23,7 +23,6 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.io.Closeable; -import org.apache.hadoop.mapred.JobConfigurable; /** * Base class for {@link Mapper} and {@link Reducer} implementations. @@ -31,7 +30,6 @@ import org.apache.hadoop.mapred.JobConfigurable; *

Provides default no-op implementations for a few methods, most non-trivial * applications need to override some of them.

*/ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class MapReduceBase implements Closeable, JobConfigurable { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapRunnable.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapRunnable.java index c989a6f32b9..7aa4f336ae5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapRunnable.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapRunnable.java @@ -30,9 +30,7 @@ import org.apache.hadoop.classification.InterfaceStability; * control on map processing e.g. multi-threaded, asynchronous mappers etc.

* * @see Mapper - * @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface MapRunnable diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java index 7c47aa91d51..e44f28b5774 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java @@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem.Statistics; -import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Mapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Mapper.java index 5e98a54f85f..2b1362c55c7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Mapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Mapper.java @@ -129,9 +129,7 @@ import org.apache.hadoop.io.compress.CompressionCodec; * @see MapReduceBase * @see MapRunnable * @see SequenceFile - * @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface Mapper extends JobConfigurable, Closeable { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java index 2a14755930b..d6f7346c63b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java @@ -21,12 +21,16 @@ package org.apache.hadoop.mapred; import java.io.IOException; import java.net.InetSocketAddress; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.yarn.conf.YarnConfiguration; +@Private +@Unstable public class Master { public enum State { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileInputFormat.java index 9b342667885..7b9dc62c2b6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileInputFormat.java @@ -36,9 +36,7 @@ import org.apache.hadoop.fs.Path; * Subclasses implement {@link #getRecordReader(InputSplit, JobConf, Reporter)} * to construct RecordReader's for MultiFileSplit's. * @see MultiFileSplit - * @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileInputFormat} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class MultiFileInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileSplit.java index 55134d61baf..8ea4f093ec6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileSplit.java @@ -18,20 +18,16 @@ package org.apache.hadoop.mapred; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.HashSet; import java.util.Set; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.BlockLocation; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.lib.CombineFileSplit; /** @@ -42,9 +38,7 @@ import org.apache.hadoop.mapred.lib.CombineFileSplit; * reading one record per file. * @see FileSplit * @see MultiFileInputFormat - * @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileSplit} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class MultiFileSplit extends CombineFileSplit { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputCommitter.java index 60fd7f99adc..8f90a87b658 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputCommitter.java @@ -56,9 +56,7 @@ import org.apache.hadoop.classification.InterfaceStability; * @see FileOutputCommitter * @see JobContext * @see TaskAttemptContext - * @deprecated Use {@link org.apache.hadoop.mapreduce.OutputCommitter} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class OutputCommitter diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputFormat.java index 342730896b9..6ea6a71afb7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputFormat.java @@ -44,9 +44,7 @@ import org.apache.hadoop.util.Progressable; * * @see RecordWriter * @see JobConf - * @deprecated Use {@link org.apache.hadoop.mapreduce.OutputFormat} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface OutputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputLogFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputLogFilter.java index 44367e4d095..363cf4ce59a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputLogFilter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputLogFilter.java @@ -29,9 +29,6 @@ import org.apache.hadoop.fs.PathFilter; * This can be used to list paths of output directory as follows: * Path[] fileList = FileUtil.stat2Paths(fs.listStatus(outDir, * new OutputLogFilter())); - * @deprecated Use - * {@link org.apache.hadoop.mapred.Utils.OutputFileUtils.OutputLogFilter} - * instead. */ @InterfaceAudience.Public @InterfaceStability.Stable diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Partitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Partitioner.java index c35e88cc8cd..1aa0ab1f92c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Partitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Partitioner.java @@ -32,9 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability; * record) is sent for reduction.

* * @see Reducer - * @deprecated Use {@link org.apache.hadoop.mapreduce.Partitioner} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface Partitioner extends JobConfigurable { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/PeriodicStatsAccumulator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/PeriodicStatsAccumulator.java index 85ee8a544e7..33eef76881c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/PeriodicStatsAccumulator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/PeriodicStatsAccumulator.java @@ -18,6 +18,9 @@ package org.apache.hadoop.mapred; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + /** * * This abstract class that represents a bucketed series of @@ -33,6 +36,8 @@ package org.apache.hadoop.mapred; * bucket and how we interpret the readings by overriding * {@code extendInternal(...)} and {@code initializeInterval()} */ +@Private +@Unstable public abstract class PeriodicStatsAccumulator { // The range of progress from 0.0D through 1.0D is divided into // count "progress segments". This object accumulates an diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ProgressSplitsBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ProgressSplitsBlock.java index d3912438527..ccc016a9b35 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ProgressSplitsBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ProgressSplitsBlock.java @@ -18,13 +18,16 @@ package org.apache.hadoop.mapred; -import java.util.List; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; /* * This object gathers the [currently four] PeriodStatset's that we * are gathering for a particular task attempt for packaging and * handling as a single object. */ +@Private +@Unstable public class ProgressSplitsBlock { final PeriodicStatsAccumulator progressWallclockTime; final PeriodicStatsAccumulator progressCPUTime; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueAclsInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueAclsInfo.java index 49c06d3cae4..639ff2a8bfd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueAclsInfo.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueAclsInfo.java @@ -20,9 +20,7 @@ package org.apache.hadoop.mapred; /** * Class to encapsulate Queue ACLs for a particular * user. - * @deprecated Use {@link org.apache.hadoop.mapreduce.QueueAclsInfo} instead */ -@Deprecated class QueueAclsInfo extends org.apache.hadoop.mapreduce.QueueAclsInfo { /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RecordReader.java index 432337b305a..0c95a147dc6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RecordReader.java @@ -19,7 +19,6 @@ package org.apache.hadoop.mapred; import java.io.IOException; -import java.io.DataInput; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Reducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Reducer.java index d51673cf58b..3fefa4bed71 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Reducer.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Reducer.java @@ -162,9 +162,7 @@ import org.apache.hadoop.io.Closeable; * @see Partitioner * @see Reporter * @see MapReduceBase - * @deprecated Use {@link org.apache.hadoop.mapreduce.Reducer} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface Reducer extends JobConfigurable, Closeable { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RunningJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RunningJob.java index 53b7188e9bf..5a11fa876ea 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RunningJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RunningJob.java @@ -34,9 +34,7 @@ import org.apache.hadoop.conf.Configuration; * progress etc.

* * @see JobClient - * @deprecated Use {@link org.apache.hadoop.mapreduce.Job} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface RunningJob { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryInputFormat.java index bfd824adf4e..1ed879e7f66 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryInputFormat.java @@ -27,20 +27,11 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.mapred.SequenceFileInputFormat; /** * InputFormat reading keys, values from SequenceFiles in binary (raw) * format. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class SequenceFileAsBinaryInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryOutputFormat.java index bd1ab2934b8..60bb16ccb03 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryOutputFormat.java @@ -23,26 +23,20 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; - -import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.CompressionType; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.ReflectionUtils; /** * An {@link OutputFormat} that writes keys, values to * {@link SequenceFile}s in binary(raw) format - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.SequenceFileAsBinaryOutputFormat} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class SequenceFileAsBinaryOutputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextInputFormat.java index 9df190a5d0c..55afa82bf7d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextInputFormat.java @@ -29,12 +29,7 @@ import org.apache.hadoop.io.Text; * except it generates SequenceFileAsTextRecordReader * which converts the input keys and values to their * String forms by calling toString() method. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileAsTextInputFormat} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class SequenceFileAsTextInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextRecordReader.java index 510483d2171..45c6c484f01 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextRecordReader.java @@ -31,11 +31,7 @@ import org.apache.hadoop.io.WritableComparable; * This class converts the input keys and values to their String forms by calling toString() * method. This class to SequenceFileAsTextInputFormat class is as LineRecordReader * class to TextInputFormat class. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileAsTextRecordReader} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class SequenceFileAsTextRecordReader diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFilter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFilter.java index e7db705a3fd..da88f2c3bcb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFilter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFilter.java @@ -29,11 +29,7 @@ import org.apache.hadoop.util.ReflectionUtils; /** * A class that allows a map/red job to work on a sample of sequence files. * The sample is decided by the filter class set by the job. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFilter} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class SequenceFileInputFilter diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java index d9ca9cc045d..9cd2da95b03 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java @@ -29,12 +29,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.MapFile; -/** An {@link InputFormat} for {@link SequenceFile}s. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat} - * instead. +/** + * An {@link InputFormat} for {@link SequenceFile}s. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class SequenceFileInputFormat extends FileInputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java index 143c86e9dea..d1040a6229c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java @@ -21,25 +21,22 @@ package org.apache.hadoop.mapred; import java.io.IOException; import java.util.Arrays; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.FileUtil; - +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.util.*; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.ReflectionUtils; -/** An {@link OutputFormat} that writes {@link SequenceFile}s. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat} - * instead. +/** + * An {@link OutputFormat} that writes {@link SequenceFile}s. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class SequenceFileOutputFormat extends FileOutputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java index 11163cd44e3..a7fb8ac3b02 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java @@ -29,7 +29,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.*; import org.apache.hadoop.util.ReflectionUtils; -/** An {@link RecordReader} for {@link SequenceFile}s. */ +/** + * An {@link RecordReader} for {@link SequenceFile}s. + */ @InterfaceAudience.Public @InterfaceStability.Stable public class SequenceFileRecordReader implements RecordReader { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java index 883b9ed871e..dc673350d96 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.mapred; -import java.io.DataInputStream; -import java.io.File; +import static org.apache.hadoop.mapred.MapTask.MAP_OUTPUT_INDEX_RECORD_LENGTH; + import java.io.IOException; import java.nio.ByteBuffer; import java.nio.LongBuffer; @@ -27,15 +27,12 @@ import java.util.zip.CheckedOutputStream; import java.util.zip.Checksum; import org.apache.hadoop.fs.ChecksumException; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.SecureIOUtils; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.PureJavaCrc32; -import org.apache.hadoop.fs.FSDataInputStream; -import static org.apache.hadoop.mapred.MapTask.MAP_OUTPUT_INDEX_RECORD_LENGTH; class SpillRecord { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java index 7e339c868c3..c74c9b9d8d6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java @@ -525,7 +525,7 @@ abstract public class Task implements Writable, Configurable { if (outputPath != null) { if ((committer instanceof FileOutputCommitter)) { FileOutputFormat.setWorkOutputPath(conf, - ((FileOutputCommitter)committer).getTempTaskOutputPath(taskContext)); + ((FileOutputCommitter)committer).getTaskAttemptPath(taskContext)); } else { FileOutputFormat.setWorkOutputPath(conf, outputPath); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContext.java index 3b6f2c00ac0..86e84798889 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContext.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContext.java @@ -22,11 +22,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.util.Progressable; -/** - * @deprecated Use {@link org.apache.hadoop.mapreduce.TaskAttemptContext} - * instead. - */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface TaskAttemptContext diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java index e278a8bfc99..60769e48284 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java @@ -22,11 +22,6 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.util.Progressable; -/** - * @deprecated Use {@link org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl} - * instead. - */ -@Deprecated @InterfaceAudience.Private @InterfaceStability.Unstable public class TaskAttemptContextImpl diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptID.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptID.java index 48756fe2125..7bb7f8a5494 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptID.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptID.java @@ -45,7 +45,6 @@ import org.apache.hadoop.mapreduce.TaskType; * @see JobID * @see TaskID */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class TaskAttemptID extends org.apache.hadoop.mapreduce.TaskAttemptID { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java index b75c96a5157..9f25339a646 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java @@ -24,10 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This is used to track task completion events on * job tracker. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.TaskCompletionEvent} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class TaskCompletionEvent diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskID.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskID.java index 036e44f7391..6234243b8ba 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskID.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskID.java @@ -47,7 +47,6 @@ import org.apache.hadoop.mapreduce.TaskType; * @see JobID * @see TaskAttemptID */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class TaskID extends org.apache.hadoop.mapreduce.TaskID { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java index 9cd2a2c05bd..4d47df9b8d8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java @@ -23,7 +23,6 @@ import java.io.BufferedReader; import java.io.DataOutputStream; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -33,13 +32,12 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.SecureIOUtils; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskReport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskReport.java index e8b7a0b85f8..1aed3e0c225 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskReport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskReport.java @@ -24,10 +24,9 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -/** A report on the state of a task. - * @deprecated Use {@link org.apache.hadoop.mapreduce.TaskReport} instead - **/ -@Deprecated +/** + * A report on the state of a task. + */ @InterfaceAudience.Public @InterfaceStability.Stable public class TaskReport extends org.apache.hadoop.mapreduce.TaskReport { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextInputFormat.java index d90a0748be9..02cffb84adb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextInputFormat.java @@ -27,13 +27,11 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.*; -/** An {@link InputFormat} for plain text files. Files are broken into lines. +/** + * An {@link InputFormat} for plain text files. Files are broken into lines. * Either linefeed or carriage-return are used to signal end of line. Keys are * the position in the file, and values are the line of text.. - * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.input.TextInputFormat} - * instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class TextInputFormat extends FileInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextOutputFormat.java index ba5ab351705..ca499e49a5a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TextOutputFormat.java @@ -34,11 +34,9 @@ import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.util.*; -/** An {@link OutputFormat} that writes plain text files. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.TextOutputFormat} instead. +/** + * An {@link OutputFormat} that writes plain text files. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class TextOutputFormat extends FileOutputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/jobcontrol/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/jobcontrol/Job.java index a2868bb6a6f..bc719d745b2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/jobcontrol/Job.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/jobcontrol/Job.java @@ -32,10 +32,6 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobID; import org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob; -/** - * @deprecated Use {@link ControlledJob} instead. - */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class Job extends ControlledJob { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/jobcontrol/JobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/jobcontrol/JobControl.java index f6ea6675fc3..4fd0a867163 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/jobcontrol/JobControl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/jobcontrol/JobControl.java @@ -26,11 +26,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob; -/** - *@deprecated Use - *{@link org.apache.hadoop.mapreduce.lib.jobcontrol.JobControl} instead - **/ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class JobControl extends diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/ArrayListBackedIterator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/ArrayListBackedIterator.java index d02d743836c..7179f11227d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/ArrayListBackedIterator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/ArrayListBackedIterator.java @@ -28,10 +28,7 @@ import org.apache.hadoop.io.Writable; * implementation uses an {@link java.util.ArrayList} to store elements * added to it, replaying them as requested. * Prefer {@link StreamBackedIterator}. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.ArrayListBackedIterator} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class ArrayListBackedIterator extends diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/ComposableInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/ComposableInputFormat.java index 1fa73b5024e..da09e7f3413 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/ComposableInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/ComposableInputFormat.java @@ -32,11 +32,7 @@ import org.apache.hadoop.mapred.Reporter; /** * Refinement of InputFormat requiring implementors to provide * ComposableRecordReader instead of RecordReader. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.ComposableInputFormat} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface ComposableInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/CompositeInputSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/CompositeInputSplit.java index 11f976662d2..9f5336f3ebc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/CompositeInputSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/CompositeInputSplit.java @@ -33,11 +33,7 @@ import org.apache.hadoop.util.ReflectionUtils; /** * This InputSplit contains a set of child InputSplits. Any InputSplit inserted * into this collection must have a public default constructor. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.CompositeInputSplit} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class CompositeInputSplit implements InputSplit { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/CompositeRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/CompositeRecordReader.java index ea5cd8f4f24..8bb5fcd2efd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/CompositeRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/CompositeRecordReader.java @@ -37,11 +37,7 @@ import org.apache.hadoop.util.ReflectionUtils; /** * A RecordReader that can effect joins of RecordReaders sharing a common key * type and partitioning. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.CompositeRecordReader} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class CompositeRecordReader< diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/InnerJoinRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/InnerJoinRecordReader.java index f9e0720707b..eef074f34d8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/InnerJoinRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/InnerJoinRecordReader.java @@ -28,11 +28,7 @@ import org.apache.hadoop.mapred.JobConf; /** * Full inner join. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.InnerJoinRecordReader} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class InnerJoinRecordReader diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/JoinRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/JoinRecordReader.java index dd0d8bd11b3..bb3bd718697 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/JoinRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/JoinRecordReader.java @@ -31,10 +31,7 @@ import org.apache.hadoop.mapred.JobConf; /** * Base class for Composite joins returning Tuples of arbitrary Writables. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.JoinRecordReader} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class JoinRecordReader diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/MultiFilterRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/MultiFilterRecordReader.java index e126da49578..9760503a280 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/MultiFilterRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/MultiFilterRecordReader.java @@ -34,10 +34,7 @@ import org.apache.hadoop.mapred.RecordReader; /** * Base class for Composite join returning values derived from multiple * sources, but generally not tuples. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.MultiFilterRecordReader} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class MultiFilterRecordReader diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/OverrideRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/OverrideRecordReader.java index 2430e731a30..1671e6e8956 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/OverrideRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/OverrideRecordReader.java @@ -34,10 +34,7 @@ import org.apache.hadoop.mapred.JobConf; * For example, override(S1,S2,S3) will prefer values * from S3 over S2, and values from S2 over S1 for all keys * emitted from all sources. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.OverrideRecordReader} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class OverrideRecordReader diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/StreamBackedIterator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/StreamBackedIterator.java index ae85c2d1ea4..0cb6df8bcdd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/StreamBackedIterator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/StreamBackedIterator.java @@ -24,11 +24,7 @@ import org.apache.hadoop.io.Writable; /** * This class provides an implementation of ResetableIterator. This * implementation uses a byte array to store elements added to it. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.StreamBackedIterator} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class StreamBackedIterator diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/TupleWritable.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/TupleWritable.java index bacb12f3f76..9fb873282cd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/TupleWritable.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/TupleWritable.java @@ -33,11 +33,7 @@ import org.apache.hadoop.io.Writable; * incompatible with, but contrary to the general case. * * @see org.apache.hadoop.io.Writable - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.TupleWritable} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class TupleWritable diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/WrappedRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/WrappedRecordReader.java index aa0cae5e1e2..5b38ba2c20e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/WrappedRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/join/WrappedRecordReader.java @@ -33,10 +33,7 @@ import org.apache.hadoop.mapred.RecordReader; * This class keeps track of the "head" key-value pair for the * provided RecordReader and keeps a store of values matching a key when * this source is participating in a join. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.join.WrappedRecordReader} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class WrappedRecordReader diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/Chain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/Chain.java index d9998124e16..57841f0d781 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/Chain.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/Chain.java @@ -35,9 +35,7 @@ import java.util.List; /** * The Chain class provides all the common functionality for the * {@link ChainMapper} and the {@link ChainReducer} classes. - * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.chain.Chain} instead */ -@Deprecated class Chain extends org.apache.hadoop.mapreduce.lib.chain.Chain { private static final String MAPPER_BY_VALUE = "chain.mapper.byValue"; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/ChainMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/ChainMapper.java index 2d781c033c3..14f040af96a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/ChainMapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/ChainMapper.java @@ -88,10 +88,7 @@ import java.io.IOException; * RunningJob job = jc.submitJob(conf); * ... * - * @deprecated - * Use {@link org.apache.hadoop.mapreduce.lib.chain.ChainMapper} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class ChainMapper implements Mapper { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/ChainReducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/ChainReducer.java index 6dd276eace5..641d82c0839 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/ChainReducer.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/ChainReducer.java @@ -88,10 +88,7 @@ import java.util.Iterator; * RunningJob job = jc.submitJob(conf); * ... * - * @deprecated - * Use {@link org.apache.hadoop.mapreduce.lib.chain.ChainReducer} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class ChainReducer implements Reducer { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java index 35dd3d7e5c6..1401fc29704 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java @@ -24,14 +24,12 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -53,10 +51,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; * Subclasses implement {@link org.apache.hadoop.mapred.InputFormat#getRecordReader(InputSplit, JobConf, Reporter)} * to construct RecordReader's for CombineFileSplit's. * @see CombineFileSplit - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat} */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class CombineFileInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReader.java index 7277650cb2a..1abaef260c5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileRecordReader.java @@ -35,10 +35,7 @@ import org.apache.hadoop.conf.Configuration; * This class allows using different RecordReaders for processing * these data chunks from different files. * @see CombineFileSplit - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader} */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class CombineFileRecordReader implements RecordReader { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileSplit.java index a23c981ac63..4628e550fdc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/CombineFileSplit.java @@ -26,11 +26,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; -/** - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.CombineFileSplit} - */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class CombineFileSplit extends diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java index 3177e9fe365..c3666d207e7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/DelegatingInputFormat.java @@ -43,10 +43,7 @@ import org.apache.hadoop.util.ReflectionUtils; * InputFormats. * * @see MultipleInputs#addInputPath(JobConf, Path, Class, Class) - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.DelegatingInputFormat} instead */ -@Deprecated @InterfaceAudience.Private @InterfaceStability.Unstable public class DelegatingInputFormat implements InputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/DelegatingMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/DelegatingMapper.java index fe1d1ca3737..1df71dc7be8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/DelegatingMapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/DelegatingMapper.java @@ -34,10 +34,7 @@ import org.apache.hadoop.util.ReflectionUtils; * mappers. * * @see MultipleInputs#addInputPath(JobConf, Path, Class, Class) - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.DelegatingMapper} instead */ -@Deprecated @InterfaceAudience.Private @InterfaceStability.Unstable public class DelegatingMapper implements Mapper { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.java index efa17a3fe7c..03335f90693 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.java @@ -65,10 +65,7 @@ import org.apache.hadoop.mapreduce.lib.fieldsel.*; * * The reducer extracts output key/value pairs in a similar manner, except that * the key is never ignored. - * @deprecated Use {@link FieldSelectionMapper} and - * {@link FieldSelectionReducer} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class FieldSelectionMapReduce diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/FilterOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/FilterOutputFormat.java index a69d9ecbca6..c2f7e613049 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/FilterOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/FilterOutputFormat.java @@ -31,10 +31,7 @@ import org.apache.hadoop.util.Progressable; /** * FilterOutputFormat is a convenience class that wraps OutputFormat. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.FilterOutputFormat} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class FilterOutputFormat implements OutputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/HashPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/HashPartitioner.java index ef62a38f666..6ac0e31fb07 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/HashPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/HashPartitioner.java @@ -23,11 +23,9 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapred.Partitioner; import org.apache.hadoop.mapred.JobConf; -/** Partition keys by their {@link Object#hashCode()}. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.partition.HashPartitioner} instead. +/** + * Partition keys by their {@link Object#hashCode()}. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class HashPartitioner implements Partitioner { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/IdentityMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/IdentityMapper.java index 82b26d05781..fe395d3ddc3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/IdentityMapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/IdentityMapper.java @@ -27,10 +27,9 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.MapReduceBase; -/** Implements the identity function, mapping inputs directly to outputs. - * @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead. +/** + * Implements the identity function, mapping inputs directly to outputs. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class IdentityMapper diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/IdentityReducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/IdentityReducer.java index b79ce6b843b..54097c80feb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/IdentityReducer.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/IdentityReducer.java @@ -29,10 +29,9 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.MapReduceBase; -/** Performs no reduction, writing all input values directly to the output. - * @deprecated Use {@link org.apache.hadoop.mapreduce.Reducer} instead. +/** + * Performs no reduction, writing all input values directly to the output. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class IdentityReducer diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/InputSampler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/InputSampler.java index 6fdef217d94..b99b0c7d6e4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/InputSampler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/InputSampler.java @@ -25,11 +25,6 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.Job; -/** - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.partition.InputSampler} - */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class InputSampler extends diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/InverseMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/InverseMapper.java index 9d1d81728e2..03d3abf4b4d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/InverseMapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/InverseMapper.java @@ -27,11 +27,9 @@ import org.apache.hadoop.mapred.Mapper; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; -/** A {@link Mapper} that swaps keys and values. - * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.map.InverseMapper} - * instead. +/** + * A {@link Mapper} that swaps keys and values. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class InverseMapper diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/KeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/KeyFieldBasedComparator.java index a4a8cb73f39..c989d77b20f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/KeyFieldBasedComparator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/KeyFieldBasedComparator.java @@ -38,11 +38,7 @@ import org.apache.hadoop.mapreduce.JobContext; * field). opts are ordering options (any of 'nr' as described above). * We assume that the fields in the key are separated by * {@link JobContext#MAP_OUTPUT_KEY_FIELD_SEPERATOR} - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.partition.KeyFieldBasedComparator} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class KeyFieldBasedComparator extends diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.java index 644a24cc3e9..c2800ead9e5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.java @@ -34,11 +34,7 @@ import org.apache.hadoop.mapred.Partitioner; * field's last character. If '.c' is omitted from pos1, it defaults to 1 * (the beginning of the field); if omitted from pos2, it defaults to 0 * (the end of the field). - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.partition.KeyFieldBasedPartitioner} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class KeyFieldBasedPartitioner extends diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/LazyOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/LazyOutputFormat.java index 39011f8e06d..a763abb3934 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/LazyOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/LazyOutputFormat.java @@ -32,10 +32,7 @@ import org.apache.hadoop.util.ReflectionUtils; /** * A Convenience class that creates output lazily. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class LazyOutputFormat extends FilterOutputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/LongSumReducer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/LongSumReducer.java index 0abd4f9c0bf..55f5113d9ca 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/LongSumReducer.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/LongSumReducer.java @@ -30,11 +30,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.io.LongWritable; -/** A {@link Reducer} that sums long values. - * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.reduce.LongSumReducer} - * instead. +/** + * A {@link Reducer} that sums long values. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class LongSumReducer extends MapReduceBase diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleInputs.java index 52e4a768193..f40c87bb164 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleInputs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleInputs.java @@ -32,10 +32,7 @@ import org.apache.hadoop.util.ReflectionUtils; /** * This class supports MapReduce jobs that have multiple input paths with * a different {@link InputFormat} and {@link Mapper} for each path - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.MultipleInputs} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class MultipleInputs { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputFormat.java index b8944f1c06b..90ce57aa36c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputFormat.java @@ -50,11 +50,7 @@ import org.apache.hadoop.util.Progressable; * * Case three: This class is used for a map only job. The job wants to use an * output file name that depends on both the keys and the input file name, - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.MultipleOutputs} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public abstract class MultipleOutputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java index 47d7fc15bf6..39e80f9a16b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleOutputs.java @@ -112,10 +112,7 @@ import java.util.*; * * } * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.MultipleOutputs} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class MultipleOutputs { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleSequenceFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleSequenceFileOutputFormat.java index 5dd2de1d7f7..63ef3c96d3a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleSequenceFileOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleSequenceFileOutputFormat.java @@ -31,10 +31,7 @@ import org.apache.hadoop.util.Progressable; /** * This class extends the MultipleOutputFormat, allowing to write the output data * to different output files in sequence file output format. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.MultipleOutputs} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class MultipleSequenceFileOutputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleTextOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleTextOutputFormat.java index 2eb1eddeb81..e7d899563dd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleTextOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultipleTextOutputFormat.java @@ -31,10 +31,7 @@ import org.apache.hadoop.util.Progressable; /** * This class extends the MultipleOutputFormat, allowing to write the output * data to different output files in Text output format. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.MultipleOutputs} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class MultipleTextOutputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java index 95620b3ccb8..456cdfab9a8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java @@ -50,9 +50,7 @@ import java.util.concurrent.*; * mapred.map.multithreadedrunner.threads property, its default * value is 10 threads. *

- * @deprecated Use {@link MultithreadedMapper} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class MultithreadedMapRunner diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/NLineInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/NLineInputFormat.java index ea022a3187c..7f4a2e5bcca 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/NLineInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/NLineInputFormat.java @@ -52,10 +52,7 @@ import org.apache.hadoop.mapred.Reporter; * a value to one map task, and key is the offset. * i.e. (k,v) is (LongWritable, Text). * The location hints will span the whole mapred cluster. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.NLineInputFormat} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class NLineInputFormat extends FileInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/NullOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/NullOutputFormat.java index 98520661278..e39ccf4800a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/NullOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/NullOutputFormat.java @@ -29,10 +29,7 @@ import org.apache.hadoop.util.Progressable; /** * Consume all outputs and put them in /dev/null. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.output.NullOutputFormat} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class NullOutputFormat implements OutputFormat { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/RegexMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/RegexMapper.java index 60cc43933ca..a95974befbe 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/RegexMapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/RegexMapper.java @@ -33,10 +33,9 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; -/** A {@link Mapper} that extracts text matching a regular expression. - * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.map.RegexMapper} +/** + * A {@link Mapper} that extracts text matching a regular expression. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class RegexMapper extends MapReduceBase diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TaggedInputSplit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TaggedInputSplit.java index 5def3c6616a..d8825a8a883 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TaggedInputSplit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TaggedInputSplit.java @@ -33,10 +33,7 @@ import org.apache.hadoop.util.ReflectionUtils; /** * An {@link InputSplit} that tags another InputSplit with extra data for use * by {@link DelegatingInputFormat}s and {@link DelegatingMapper}s. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.input.TaggedInputSplit} instead */ -@Deprecated class TaggedInputSplit implements Configurable, InputSplit { private Class inputSplitClass; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TokenCountMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TokenCountMapper.java index ac0e16d9f4c..8e884cee8bb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TokenCountMapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TokenCountMapper.java @@ -31,12 +31,10 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; -/** A {@link Mapper} that maps text values into pairs. Uses +/** + * A {@link Mapper} that maps text values into pairs. Uses * {@link StringTokenizer} to break text into tokens. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper} instead. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class TokenCountMapper extends MapReduceBase diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TotalOrderPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TotalOrderPartitioner.java index 1eed74ae127..14e0962da2c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TotalOrderPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/TotalOrderPartitioner.java @@ -28,10 +28,7 @@ import org.apache.hadoop.mapred.Partitioner; /** * Partitioner effecting a total order by reading split points from * an externally generated source. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner} */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class TotalOrderPartitioner,V> diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java index 328e2c46233..ac791c177f4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/DoubleValueSum.java @@ -24,11 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This class implements a value aggregator that sums up a sequence of double * values. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.DoubleValueSum} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class DoubleValueSum diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java index 01bb1d10f80..0662d7e3910 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMax.java @@ -25,11 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This class implements a value aggregator that maintain the maximum of * a sequence of long values. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.LongValueMax} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class LongValueMax diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java index 71a098274c0..1e8b51a2e78 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueMin.java @@ -24,11 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This class implements a value aggregator that maintain the minimum of * a sequence of long values. - * - *@deprecated Use - *{@link org.apache.hadoop.mapreduce.lib.aggregate.LongValueMin} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class LongValueMin diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java index 9f3efe82fc3..ad38a0fb01c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/LongValueSum.java @@ -24,11 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This class implements a value aggregator that sums up * a sequence of long values. - * - *@deprecated Use - *{@link org.apache.hadoop.mapreduce.lib.aggregate.LongValueSum} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class LongValueSum diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java index 2bdc70f99d9..eabcac87143 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMax.java @@ -24,11 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This class implements a value aggregator that maintain the biggest of * a sequence of strings. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.StringValueMax} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class StringValueMax diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java index 4990fdf5ca0..e3acc014971 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/StringValueMin.java @@ -24,11 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This class implements a value aggregator that maintain the smallest of * a sequence of strings. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.StringValueMin} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class StringValueMin diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java index 735dc4d754e..f59e18f3699 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/UniqValueCount.java @@ -23,11 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This class implements a value aggregator that dedupes a sequence of objects. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.UniqValueCount} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class UniqValueCount diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java index 53b564c85c5..cfddfda4ded 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/UserDefinedValueAggregatorDescriptor.java @@ -23,16 +23,13 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.mapred.JobConf; /** - * This class implements a wrapper for a user defined value aggregator descriptor. - * It servs two functions: One is to create an object of ValueAggregatorDescriptor from the - * name of a user defined class that may be dynamically loaded. The other is to - * deligate inviokations of generateKeyValPairs function to the created object. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.UserDefinedValueAggregatorDescriptor} - * instead + * This class implements a wrapper for a user defined value aggregator + * descriptor. + * It serves two functions: One is to create an object of + * ValueAggregatorDescriptor from the name of a user defined class that may be + * dynamically loaded. The other is to delegate invocations of + * generateKeyValPairs function to the created object. */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class UserDefinedValueAggregatorDescriptor extends org.apache.hadoop. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java index a61d61d71be..064a720d9e2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregator.java @@ -23,11 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability; /** * This interface defines the minimal protocol for value aggregators. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.ValueAggregator} instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface ValueAggregator extends diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java index 694353b0869..b10ac583505 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorBaseDescriptor.java @@ -28,12 +28,7 @@ import org.apache.hadoop.mapred.JobConf; /** * This class implements the common functionalities of * the subclasses of ValueAggregatorDescriptor class. - * - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.ValueAggregatorBaseDescriptor} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class ValueAggregatorBaseDescriptor extends org.apache.hadoop.mapreduce. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java index 4b71aae36ab..d250cadcf6a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorCombiner.java @@ -32,11 +32,7 @@ import org.apache.hadoop.mapred.Reporter; /** * This class implements the generic combiner of Aggregate. - * @deprecated Use - * {@link org.apache.hadoop.mapreduce.lib.aggregate.ValueAggregatorCombiner} - * instead */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class ValueAggregatorCombiner diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/db/DBOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/db/DBOutputFormat.java index 549dd3237ff..6c2ecf772f5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/db/DBOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/db/DBOutputFormat.java @@ -31,15 +31,10 @@ import org.apache.hadoop.mapred.OutputFormat; import org.apache.hadoop.mapred.RecordWriter; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import org.apache.hadoop.util.Progressable; -/** - * @deprecated Use org.apache.hadoop.mapreduce.lib.db.DBOutputFormat instead - */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public class DBOutputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/db/DBWritable.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/db/DBWritable.java index df8a0788dd9..1e38e4de10c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/db/DBWritable.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/lib/db/DBWritable.java @@ -20,11 +20,6 @@ package org.apache.hadoop.mapred.lib.db; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -/** - * @deprecated - * Use {@link org.apache.hadoop.mapreduce.lib.db.DBWritable} instead - */ -@Deprecated @InterfaceAudience.Public @InterfaceStability.Stable public interface DBWritable diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java index 0a108d73b63..8d4d2592373 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java @@ -18,7 +18,6 @@ package org.apache.hadoop.mapred.pipes; -import java.io.BufferedInputStream; import java.io.File; import java.io.IOException; import java.net.ServerSocket; @@ -27,14 +26,12 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Random; + import javax.crypto.SecretKey; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; @@ -50,6 +47,8 @@ import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.TaskAttemptID; import org.apache.hadoop.mapred.TaskLog; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.mapreduce.security.SecureShuffleUtils; import org.apache.hadoop.mapreduce.security.TokenCache; import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java index ecdce749abd..ebfb1845377 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/BinaryProtocol.java @@ -18,14 +18,20 @@ package org.apache.hadoop.mapred.pipes; -import java.io.*; +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.FileOutputStream; +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.net.Socket; import java.util.ArrayList; import java.util.List; import java.util.Map; -import javax.crypto.SecretKey; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.io.BytesWritable; @@ -36,8 +42,6 @@ import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.security.SecureShuffleUtils; -import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; import org.apache.hadoop.util.StringUtils; /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/OutputHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/OutputHandler.java index 04cf21a9496..2d6850f4c41 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/OutputHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/OutputHandler.java @@ -19,9 +19,7 @@ package org.apache.hadoop.mapred.pipes; import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import org.apache.hadoop.io.FloatWritable; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java index 4c0d940609c..ddc3ae80172 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java @@ -25,7 +25,6 @@ import java.net.URL; import java.net.URLClassLoader; import java.security.AccessController; import java.security.PrivilegedAction; -import java.util.Iterator; import java.util.StringTokenizer; import org.apache.commons.cli.BasicParser; @@ -41,8 +40,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; @@ -59,6 +56,8 @@ import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.lib.HashPartitioner; import org.apache.hadoop.mapred.lib.LazyOutputFormat; import org.apache.hadoop.mapred.lib.NullOutputFormat; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.filecache.DistributedCache; import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.Tool; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java index f63352b4dd1..4038f65cd46 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java @@ -369,6 +369,12 @@ class JobSubmitter { conf.set(toFullPropertyName(queue, QueueACL.ADMINISTER_JOBS.getAclName()), acl.getAclString()); + // removing jobtoken referrals before copying the jobconf to HDFS + // as the tasks don't need this setting, actually they may break + // because of it if present as the referral will point to a + // different job. + TokenCache.cleanUpTokenReferral(conf); + // Write job file to submit dir writeConf(conf, submitJobFile); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/OutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/OutputCommitter.java index 819c32baa9f..7006eba2d58 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/OutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/OutputCommitter.java @@ -51,17 +51,21 @@ import org.apache.hadoop.classification.InterfaceStability; * Discard the task commit. * * + * The methods in this class can be called from several different processes and + * from several different contexts. It is important to know which process and + * which context each is called from. Each method should be marked accordingly + * in its documentation. * * @see org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter * @see JobContext * @see TaskAttemptContext - * */ @InterfaceAudience.Public @InterfaceStability.Stable public abstract class OutputCommitter { /** - * For the framework to setup the job output during initialization + * For the framework to setup the job output during initialization. This is + * called from the application master process for the entire job. * * @param jobContext Context of the job whose output is being written. * @throws IOException if temporary output could not be created @@ -69,11 +73,12 @@ public abstract class OutputCommitter { public abstract void setupJob(JobContext jobContext) throws IOException; /** - * For cleaning up the job's output after job completion + * For cleaning up the job's output after job completion. This is called + * from the application master process for the entire job. * * @param jobContext Context of the job whose output is being written. * @throws IOException - * @deprecated Use {@link #commitJob(JobContext)} or + * @deprecated Use {@link #commitJob(JobContext)} and * {@link #abortJob(JobContext, JobStatus.State)} instead. */ @Deprecated @@ -81,7 +86,8 @@ public abstract class OutputCommitter { /** * For committing job's output after successful job completion. Note that this - * is invoked for jobs with final runstate as SUCCESSFUL. + * is invoked for jobs with final runstate as SUCCESSFUL. This is called + * from the application master process for the entire job. * * @param jobContext Context of the job whose output is being written. * @throws IOException @@ -94,7 +100,8 @@ public abstract class OutputCommitter { /** * For aborting an unsuccessful job's output. Note that this is invoked for * jobs with final runstate as {@link JobStatus.State#FAILED} or - * {@link JobStatus.State#KILLED}. + * {@link JobStatus.State#KILLED}. This is called from the application + * master process for the entire job. * * @param jobContext Context of the job whose output is being written. * @param state final runstate of the job @@ -106,7 +113,8 @@ public abstract class OutputCommitter { } /** - * Sets up output for the task. + * Sets up output for the task. This is called from each individual task's + * process that will output to HDFS, and it is called just for that task. * * @param taskContext Context of the task whose output is being written. * @throws IOException @@ -115,7 +123,9 @@ public abstract class OutputCommitter { throws IOException; /** - * Check whether task needs a commit + * Check whether task needs a commit. This is called from each individual + * task's process that will output to HDFS, and it is called just for that + * task. * * @param taskContext * @return true/false @@ -125,18 +135,23 @@ public abstract class OutputCommitter { throws IOException; /** - * To promote the task's temporary output to final output location - * - * The task's output is moved to the job's output directory. + * To promote the task's temporary output to final output location. + * If {@link #needsTaskCommit(TaskAttemptContext)} returns true and this + * task is the task that the AM determines finished first, this method + * is called to commit an individual task's output. This is to mark + * that tasks output as complete, as {@link #commitJob(JobContext)} will + * also be called later on if the entire job finished successfully. This + * is called from a task's process. * * @param taskContext Context of the task whose output is being written. - * @throws IOException if commit is not + * @throws IOException if commit is not successful. */ public abstract void commitTask(TaskAttemptContext taskContext) throws IOException; /** - * Discard the task output + * Discard the task output. This is called from a task's process to clean + * up a single task's output that can not yet been committed. * * @param taskContext * @throws IOException @@ -164,7 +179,8 @@ public abstract class OutputCommitter { * The retry-count for the job will be passed via the * {@link MRJobConfig#APPLICATION_ATTEMPT_ID} key in * {@link TaskAttemptContext#getConfiguration()} for the - * OutputCommitter. + * OutputCommitter. This is called from the application master + * process, but it is called individually for each task. * * If an exception is thrown the task will be attempted again. * diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java index c9be77c29d8..aa1089f1db1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java @@ -24,8 +24,11 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -52,9 +55,13 @@ import org.apache.hadoop.yarn.api.records.ContainerId; @InterfaceStability.Unstable public class JobHistoryParser { + private static final Log LOG = LogFactory.getLog(JobHistoryParser.class); + private final FSDataInputStream in; - JobInfo info = null; + private JobInfo info = null; + private IOException parseException = null; + /** * Create a job history parser for the given history file using the * given file system @@ -91,30 +98,58 @@ public class JobHistoryParser { * The first invocation will populate the object, subsequent calls * will return the already parsed object. * The input stream is closed on return + * + * This api ignores partial records and stops parsing on encountering one. + * {@link #getParseException()} can be used to fetch the exception, if any. + * * @return The populated jobInfo object * @throws IOException + * @see #getParseException() */ public synchronized JobInfo parse() throws IOException { + return parse(new EventReader(in)); + } + + /** + * Only used for unit tests. + */ + @Private + public synchronized JobInfo parse(EventReader reader) throws IOException { if (info != null) { return info; } - EventReader reader = new EventReader(in); - - HistoryEvent event; info = new JobInfo(); + + int eventCtr = 0; + HistoryEvent event; try { while ((event = reader.getNextEvent()) != null) { handleEvent(event); - } + ++eventCtr; + } + } catch (IOException ioe) { + LOG.info("Caught exception parsing history file after " + eventCtr + + " events", ioe); + parseException = ioe; } finally { in.close(); } return info; } - private void handleEvent(HistoryEvent event) throws IOException { + /** + * Get the parse exception, if any. + * + * @return the parse exception, if any + * @see #parse() + */ + public synchronized IOException getParseException() { + return parseException; + } + + private void handleEvent(HistoryEvent event) { EventType type = event.getEventType(); switch (type) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java index ccd32e0c1bd..0845f153504 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java @@ -19,16 +19,16 @@ package org.apache.hadoop.mapreduce.lib.output; import java.io.IOException; -import java.net.URI; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability; -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.PathFilter; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.MRJobConfig; @@ -37,41 +37,239 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; /** An {@link OutputCommitter} that commits files specified - * in job output directory i.e. ${mapreduce.output.fileoutputformat.outputdir}. + * in job output directory i.e. ${mapreduce.output.fileoutputformat.outputdir}. **/ @InterfaceAudience.Public @InterfaceStability.Stable public class FileOutputCommitter extends OutputCommitter { - private static final Log LOG = LogFactory.getLog(FileOutputCommitter.class); - /** - * Temporary directory name + /** + * Name of directory where pending data is placed. Data that has not been + * committed yet. */ - protected static final String TEMP_DIR_NAME = "_temporary"; + public static final String PENDING_DIR_NAME = "_temporary"; public static final String SUCCEEDED_FILE_NAME = "_SUCCESS"; - static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = + public static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = "mapreduce.fileoutputcommitter.marksuccessfuljobs"; - private FileSystem outputFileSystem = null; private Path outputPath = null; private Path workPath = null; /** * Create a file output committer - * @param outputPath the job's output path + * @param outputPath the job's output path, or null if you want the output + * committer to act as a noop. * @param context the task's context * @throws IOException */ public FileOutputCommitter(Path outputPath, TaskAttemptContext context) throws IOException { + this(outputPath, (JobContext)context); if (outputPath != null) { - this.outputPath = outputPath; - outputFileSystem = outputPath.getFileSystem(context.getConfiguration()); - workPath = new Path(outputPath, - getTaskAttemptBaseDirName(context)) - .makeQualified(outputFileSystem); + workPath = getTaskAttemptPath(context, outputPath); } } + + /** + * Create a file output committer + * @param outputPath the job's output path, or null if you want the output + * committer to act as a noop. + * @param context the task's context + * @throws IOException + */ + @Private + public FileOutputCommitter(Path outputPath, + JobContext context) throws IOException { + if (outputPath != null) { + FileSystem fs = outputPath.getFileSystem(context.getConfiguration()); + this.outputPath = fs.makeQualified(outputPath); + } + } + + /** + * @return the path where final output of the job should be placed. This + * could also be considered the committed application attempt path. + */ + private Path getOutputPath() { + return this.outputPath; + } + + /** + * @return true if we have an output path set, else false. + */ + private boolean hasOutputPath() { + return this.outputPath != null; + } + + /** + * @return the path where the output of pending job attempts are + * stored. + */ + private Path getPendingJobAttemptsPath() { + return getPendingJobAttemptsPath(getOutputPath()); + } + + /** + * Get the location of pending job attempts. + * @param out the base output directory. + * @return the location of pending job attempts. + */ + private static Path getPendingJobAttemptsPath(Path out) { + return new Path(out, PENDING_DIR_NAME); + } + + /** + * Get the Application Attempt Id for this job + * @param context the context to look in + * @return the Application Attempt Id for a given job. + */ + private static int getAppAttemptId(JobContext context) { + return context.getConfiguration().getInt( + MRJobConfig.APPLICATION_ATTEMPT_ID, 0); + } + + /** + * Compute the path where the output of a given job attempt will be placed. + * @param context the context of the job. This is used to get the + * application attempt id. + * @return the path to store job attempt data. + */ + public Path getJobAttemptPath(JobContext context) { + return getJobAttemptPath(context, getOutputPath()); + } + + /** + * Compute the path where the output of a given job attempt will be placed. + * @param context the context of the job. This is used to get the + * application attempt id. + * @param out the output path to place these in. + * @return the path to store job attempt data. + */ + public static Path getJobAttemptPath(JobContext context, Path out) { + return getJobAttemptPath(getAppAttemptId(context), out); + } + + /** + * Compute the path where the output of a given job attempt will be placed. + * @param appAttemptId the ID of the application attempt for this job. + * @return the path to store job attempt data. + */ + private Path getJobAttemptPath(int appAttemptId) { + return getJobAttemptPath(appAttemptId, getOutputPath()); + } + + /** + * Compute the path where the output of a given job attempt will be placed. + * @param appAttemptId the ID of the application attempt for this job. + * @return the path to store job attempt data. + */ + private static Path getJobAttemptPath(int appAttemptId, Path out) { + return new Path(getPendingJobAttemptsPath(out), String.valueOf(appAttemptId)); + } + + /** + * Compute the path where the output of pending task attempts are stored. + * @param context the context of the job with pending tasks. + * @return the path where the output of pending task attempts are stored. + */ + private Path getPendingTaskAttemptsPath(JobContext context) { + return getPendingTaskAttemptsPath(context, getOutputPath()); + } + + /** + * Compute the path where the output of pending task attempts are stored. + * @param context the context of the job with pending tasks. + * @return the path where the output of pending task attempts are stored. + */ + private static Path getPendingTaskAttemptsPath(JobContext context, Path out) { + return new Path(getJobAttemptPath(context, out), PENDING_DIR_NAME); + } + + /** + * Compute the path where the output of a task attempt is stored until + * that task is committed. + * + * @param context the context of the task attempt. + * @return the path where a task attempt should be stored. + */ + public Path getTaskAttemptPath(TaskAttemptContext context) { + return new Path(getPendingTaskAttemptsPath(context), + String.valueOf(context.getTaskAttemptID())); + } + + /** + * Compute the path where the output of a task attempt is stored until + * that task is committed. + * + * @param context the context of the task attempt. + * @param out The output path to put things in. + * @return the path where a task attempt should be stored. + */ + public static Path getTaskAttemptPath(TaskAttemptContext context, Path out) { + return new Path(getPendingTaskAttemptsPath(context, out), + String.valueOf(context.getTaskAttemptID())); + } + + /** + * Compute the path where the output of a committed task is stored until + * the entire job is committed. + * @param context the context of the task attempt + * @return the path where the output of a committed task is stored until + * the entire job is committed. + */ + public Path getCommittedTaskPath(TaskAttemptContext context) { + return getCommittedTaskPath(getAppAttemptId(context), context); + } + + public static Path getCommittedTaskPath(TaskAttemptContext context, Path out) { + return getCommittedTaskPath(getAppAttemptId(context), context, out); + } + + /** + * Compute the path where the output of a committed task is stored until the + * entire job is committed for a specific application attempt. + * @param appAttemptId the id of the application attempt to use + * @param context the context of any task. + * @return the path where the output of a committed task is stored. + */ + private Path getCommittedTaskPath(int appAttemptId, TaskAttemptContext context) { + return new Path(getJobAttemptPath(appAttemptId), + String.valueOf(context.getTaskAttemptID().getTaskID())); + } + + private static Path getCommittedTaskPath(int appAttemptId, TaskAttemptContext context, Path out) { + return new Path(getJobAttemptPath(appAttemptId, out), + String.valueOf(context.getTaskAttemptID().getTaskID())); + } + + private static class CommittedTaskFilter implements PathFilter { + @Override + public boolean accept(Path path) { + return !PENDING_DIR_NAME.equals(path.getName()); + } + } + + /** + * Get a list of all paths where output from committed tasks are stored. + * @param context the context of the current job + * @return the list of these Paths/FileStatuses. + * @throws IOException + */ + private FileStatus[] getAllCommittedTaskPaths(JobContext context) + throws IOException { + Path jobAttemptPath = getJobAttemptPath(context); + FileSystem fs = jobAttemptPath.getFileSystem(context.getConfiguration()); + return fs.listStatus(jobAttemptPath, new CommittedTaskFilter()); + } + + /** + * Get the directory that the task should write results into. + * @return the work directory + * @throws IOException + */ + public Path getWorkPath() throws IOException { + return workPath; + } /** * Create the temporary directory that is the root of all of the task @@ -79,116 +277,103 @@ public class FileOutputCommitter extends OutputCommitter { * @param context the job's context */ public void setupJob(JobContext context) throws IOException { - if (outputPath != null) { - Path tmpDir = new Path(outputPath, getJobAttemptBaseDirName(context) + - Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME); - FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration()); - if (!fileSys.mkdirs(tmpDir)) { - LOG.error("Mkdirs failed to create " + tmpDir.toString()); + if (hasOutputPath()) { + Path pendingJobAttemptsPath = getPendingJobAttemptsPath(); + FileSystem fs = pendingJobAttemptsPath.getFileSystem( + context.getConfiguration()); + if (!fs.mkdirs(pendingJobAttemptsPath)) { + LOG.error("Mkdirs failed to create " + pendingJobAttemptsPath); } - } - } - - // True if the job requires output.dir marked on successful job. - // Note that by default it is set to true. - private boolean shouldMarkOutputDir(Configuration conf) { - return conf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true); - } - - // Create a _success file in the job's output dir - private void markOutputDirSuccessful(MRJobConfig context) throws IOException { - if (outputPath != null) { - // create a file in the output folder to mark the job completion - Path filePath = new Path(outputPath, SUCCEEDED_FILE_NAME); - outputFileSystem.create(filePath).close(); + } else { + LOG.warn("Output Path is null in setupJob()"); } } /** - * Move all job output to the final place. + * The job has completed so move all committed tasks to the final output dir. * Delete the temporary directory, including all of the work directories. * Create a _SUCCESS file to make it as successful. * @param context the job's context */ public void commitJob(JobContext context) throws IOException { - if (outputPath != null) { - //delete the task temp directory from the current jobtempdir - Path tmpDir = new Path(outputPath, getJobAttemptBaseDirName(context) + - Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME); - FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration()); - if (fileSys.exists(tmpDir)) { - fileSys.delete(tmpDir, true); - } else { - LOG.warn("Task temp dir could not be deleted " + tmpDir); + if (hasOutputPath()) { + Path finalOutput = getOutputPath(); + FileSystem fs = finalOutput.getFileSystem(context.getConfiguration()); + for(FileStatus stat: getAllCommittedTaskPaths(context)) { + mergePaths(fs, stat, finalOutput); } - //move the job output to final place - Path jobOutputPath = - new Path(outputPath, getJobAttemptBaseDirName(context)); - moveJobOutputs(outputFileSystem, jobOutputPath, outputPath, jobOutputPath); - // delete the _temporary folder and create a _done file in the o/p folder cleanupJob(context); - if (shouldMarkOutputDir(context.getConfiguration())) { - markOutputDirSuccessful(context); + // True if the job requires output.dir marked on successful job. + // Note that by default it is set to true. + if (context.getConfiguration().getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true)) { + Path markerPath = new Path(outputPath, SUCCEEDED_FILE_NAME); + fs.create(markerPath).close(); } + } else { + LOG.warn("Output Path is null in commitJob()"); } } /** - * Move job output to final location - * @param fs Filesystem handle - * @param origJobOutputPath The original location of the job output - * Required to generate the relative path for correct moving of data. - * @param finalOutputDir The final output directory to which the job output - * needs to be moved - * @param jobOutput The current job output directory being moved - * @throws IOException + * Merge two paths together. Anything in from will be moved into to, if there + * are any name conflicts while merging the files or directories in from win. + * @param fs the File System to use + * @param from the path data is coming from. + * @param to the path data is going to. + * @throws IOException on any error */ - private void moveJobOutputs(FileSystem fs, final Path origJobOutputPath, - Path finalOutputDir, Path jobOutput) throws IOException { - LOG.debug("Told to move job output from " + jobOutput - + " to " + finalOutputDir + - " and orig job output path is " + origJobOutputPath); - if (fs.isFile(jobOutput)) { - Path finalOutputPath = - getFinalPath(finalOutputDir, jobOutput, origJobOutputPath); - if (!fs.rename(jobOutput, finalOutputPath)) { - if (!fs.delete(finalOutputPath, true)) { - throw new IOException("Failed to delete earlier output of job"); - } - if (!fs.rename(jobOutput, finalOutputPath)) { - throw new IOException("Failed to save output of job"); - } - } - LOG.debug("Moved job output file from " + jobOutput + " to " + - finalOutputPath); - } else if (fs.getFileStatus(jobOutput).isDirectory()) { - LOG.debug("Job output file " + jobOutput + " is a dir"); - FileStatus[] paths = fs.listStatus(jobOutput); - Path finalOutputPath = - getFinalPath(finalOutputDir, jobOutput, origJobOutputPath); - fs.mkdirs(finalOutputPath); - LOG.debug("Creating dirs along job output path " + finalOutputPath); - if (paths != null) { - for (FileStatus path : paths) { - moveJobOutputs(fs, origJobOutputPath, finalOutputDir, path.getPath()); - } - } - } + private static void mergePaths(FileSystem fs, final FileStatus from, + final Path to) + throws IOException { + LOG.debug("Merging data from "+from+" to "+to); + if(from.isFile()) { + if(fs.exists(to)) { + if(!fs.delete(to, true)) { + throw new IOException("Failed to delete "+to); + } + } + + if(!fs.rename(from.getPath(), to)) { + throw new IOException("Failed to rename "+from+" to "+to); + } + } else if(from.isDirectory()) { + if(fs.exists(to)) { + FileStatus toStat = fs.getFileStatus(to); + if(!toStat.isDirectory()) { + if(!fs.delete(to, true)) { + throw new IOException("Failed to delete "+to); + } + if(!fs.rename(from.getPath(), to)) { + throw new IOException("Failed to rename "+from+" to "+to); + } + } else { + //It is a directory so merge everything in the directories + for(FileStatus subFrom: fs.listStatus(from.getPath())) { + Path subTo = new Path(to, subFrom.getPath().getName()); + mergePaths(fs, subFrom, subTo); + } + } + } else { + //it does not exist just rename + if(!fs.rename(from.getPath(), to)) { + throw new IOException("Failed to rename "+from+" to "+to); + } + } + } } @Override @Deprecated public void cleanupJob(JobContext context) throws IOException { - if (outputPath != null) { - Path tmpDir = new Path(outputPath, FileOutputCommitter.TEMP_DIR_NAME); - FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration()); - if (fileSys.exists(tmpDir)) { - fileSys.delete(tmpDir, true); - } + if (hasOutputPath()) { + Path pendingJobAttemptsPath = getPendingJobAttemptsPath(); + FileSystem fs = pendingJobAttemptsPath + .getFileSystem(context.getConfiguration()); + fs.delete(pendingJobAttemptsPath, true); } else { - LOG.warn("Output Path is null in cleanup"); + LOG.warn("Output Path is null in cleanupJob()"); } } @@ -217,69 +402,40 @@ public class FileOutputCommitter extends OutputCommitter { * Move the files from the work directory to the job output directory * @param context the task context */ + @Override public void commitTask(TaskAttemptContext context) throws IOException { - TaskAttemptID attemptId = context.getTaskAttemptID(); - if (workPath != null) { - context.progress(); - if (outputFileSystem.exists(workPath)) { - // Move the task outputs to the current job attempt output dir - Path jobOutputPath = - new Path(outputPath, getJobAttemptBaseDirName(context)); - moveTaskOutputs(context, outputFileSystem, jobOutputPath, workPath); - // Delete the temporary task-specific output directory - if (!outputFileSystem.delete(workPath, true)) { - LOG.warn("Failed to delete the temporary output" + - " directory of task: " + attemptId + " - " + workPath); - } - LOG.info("Saved output of task '" + attemptId + "' to " + - jobOutputPath); - } - } + commitTask(context, null); } - /** - * Move all of the files from the work directory to the final output - * @param context the task context - * @param fs the output file system - * @param jobOutputDir the final output direcotry - * @param taskOutput the work path - * @throws IOException - */ - private void moveTaskOutputs(TaskAttemptContext context, - FileSystem fs, - Path jobOutputDir, - Path taskOutput) + @Private + public void commitTask(TaskAttemptContext context, Path taskAttemptPath) throws IOException { TaskAttemptID attemptId = context.getTaskAttemptID(); - context.progress(); - LOG.debug("Told to move taskoutput from " + taskOutput - + " to " + jobOutputDir); - if (fs.isFile(taskOutput)) { - Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput, - workPath); - if (!fs.rename(taskOutput, finalOutputPath)) { - if (!fs.delete(finalOutputPath, true)) { - throw new IOException("Failed to delete earlier output of task: " + - attemptId); - } - if (!fs.rename(taskOutput, finalOutputPath)) { - throw new IOException("Failed to save output of task: " + - attemptId); - } + if (hasOutputPath()) { + context.progress(); + if(taskAttemptPath == null) { + taskAttemptPath = getTaskAttemptPath(context); } - LOG.debug("Moved " + taskOutput + " to " + finalOutputPath); - } else if(fs.getFileStatus(taskOutput).isDirectory()) { - LOG.debug("Taskoutput " + taskOutput + " is a dir"); - FileStatus[] paths = fs.listStatus(taskOutput); - Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput, workPath); - fs.mkdirs(finalOutputPath); - LOG.debug("Creating dirs along path " + finalOutputPath); - if (paths != null) { - for (FileStatus path : paths) { - moveTaskOutputs(context, fs, jobOutputDir, path.getPath()); + Path committedTaskPath = getCommittedTaskPath(context); + FileSystem fs = taskAttemptPath.getFileSystem(context.getConfiguration()); + if (fs.exists(taskAttemptPath)) { + if(fs.exists(committedTaskPath)) { + if(!fs.delete(committedTaskPath, true)) { + throw new IOException("Could not delete " + committedTaskPath); + } } + if(!fs.rename(taskAttemptPath, committedTaskPath)) { + throw new IOException("Could not rename " + taskAttemptPath + " to " + + committedTaskPath); + } + LOG.info("Saved output of task '" + attemptId + "' to " + + committedTaskPath); + } else { + LOG.warn("No Output found for " + attemptId); } + } else { + LOG.warn("Output Path is null in commitTask()"); } } @@ -289,38 +445,22 @@ public class FileOutputCommitter extends OutputCommitter { */ @Override public void abortTask(TaskAttemptContext context) throws IOException { - if (workPath != null) { - context.progress(); - outputFileSystem.delete(workPath, true); - } + abortTask(context, null); } - /** - * Find the final name of a given output file, given the job output directory - * and the work directory. - * @param jobOutputDir the job's output directory - * @param taskOutput the specific task output file - * @param taskOutputPath the job's work directory - * @return the final path for the specific output file - * @throws IOException - */ - private Path getFinalPath(Path jobOutputDir, Path taskOutput, - Path taskOutputPath) throws IOException { - URI taskOutputUri = taskOutput.makeQualified(outputFileSystem.getUri(), - outputFileSystem.getWorkingDirectory()).toUri(); - URI taskOutputPathUri = - taskOutputPath.makeQualified( - outputFileSystem.getUri(), - outputFileSystem.getWorkingDirectory()).toUri(); - URI relativePath = taskOutputPathUri.relativize(taskOutputUri); - if (taskOutputUri == relativePath) { - throw new IOException("Can not get the relative path: base = " + - taskOutputPathUri + " child = " + taskOutputUri); - } - if (relativePath.getPath().length() > 0) { - return new Path(jobOutputDir, relativePath.getPath()); + @Private + public void abortTask(TaskAttemptContext context, Path taskAttemptPath) throws IOException { + if (hasOutputPath()) { + context.progress(); + if(taskAttemptPath == null) { + taskAttemptPath = getTaskAttemptPath(context); + } + FileSystem fs = taskAttemptPath.getFileSystem(context.getConfiguration()); + if(!fs.delete(taskAttemptPath, true)) { + LOG.warn("Could not delete "+taskAttemptPath); + } } else { - return jobOutputDir; + LOG.warn("Output Path is null in abortTask()"); } } @@ -331,16 +471,20 @@ public class FileOutputCommitter extends OutputCommitter { @Override public boolean needsTaskCommit(TaskAttemptContext context ) throws IOException { - return workPath != null && outputFileSystem.exists(workPath); + return needsTaskCommit(context, null); } - /** - * Get the directory that the task should write results into - * @return the work directory - * @throws IOException - */ - public Path getWorkPath() throws IOException { - return workPath; + @Private + public boolean needsTaskCommit(TaskAttemptContext context, Path taskAttemptPath + ) throws IOException { + if(hasOutputPath()) { + if(taskAttemptPath == null) { + taskAttemptPath = getTaskAttemptPath(context); + } + FileSystem fs = taskAttemptPath.getFileSystem(context.getConfiguration()); + return fs.exists(taskAttemptPath); + } + return false; } @Override @@ -352,43 +496,35 @@ public class FileOutputCommitter extends OutputCommitter { public void recoverTask(TaskAttemptContext context) throws IOException { context.progress(); - Path jobOutputPath = - new Path(outputPath, getJobAttemptBaseDirName(context)); - int previousAttempt = - context.getConfiguration().getInt( - MRJobConfig.APPLICATION_ATTEMPT_ID, 0) - 1; + TaskAttemptID attemptId = context.getTaskAttemptID(); + int previousAttempt = getAppAttemptId(context) - 1; if (previousAttempt < 0) { throw new IOException ("Cannot recover task output for first attempt..."); } - - Path pathToRecover = - new Path(outputPath, getJobAttemptBaseDirName(previousAttempt)); - LOG.debug("Trying to recover task from " + pathToRecover - + " into " + jobOutputPath); - if (outputFileSystem.exists(pathToRecover)) { - // Move the task outputs to their final place - moveJobOutputs(outputFileSystem, - pathToRecover, jobOutputPath, pathToRecover); - LOG.info("Saved output of job to " + jobOutputPath); + + Path committedTaskPath = getCommittedTaskPath(context); + Path previousCommittedTaskPath = getCommittedTaskPath( + previousAttempt, context); + FileSystem fs = committedTaskPath.getFileSystem(context.getConfiguration()); + + LOG.debug("Trying to recover task from " + previousCommittedTaskPath + + " into " + committedTaskPath); + if (fs.exists(previousCommittedTaskPath)) { + if(fs.exists(committedTaskPath)) { + if(!fs.delete(committedTaskPath, true)) { + throw new IOException("Could not delete "+committedTaskPath); + } + } + //Rename can fail if the parent directory does not yet exist. + Path committedParent = committedTaskPath.getParent(); + fs.mkdirs(committedParent); + if(!fs.rename(previousCommittedTaskPath, committedTaskPath)) { + throw new IOException("Could not rename " + previousCommittedTaskPath + + " to " + committedTaskPath); + } + LOG.info("Saved output of " + attemptId + " to " + committedTaskPath); + } else { + LOG.warn(attemptId+" had no output to recover."); } } - - protected static String getJobAttemptBaseDirName(JobContext context) { - int appAttemptId = - context.getConfiguration().getInt( - MRJobConfig.APPLICATION_ATTEMPT_ID, 0); - return getJobAttemptBaseDirName(appAttemptId); - } - - protected static String getJobAttemptBaseDirName(int appAttemptId) { - return FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + - + appAttemptId; - } - - protected static String getTaskAttemptBaseDirName( - TaskAttemptContext context) { - return getJobAttemptBaseDirName(context) + Path.SEPARATOR + - FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + - "_" + context.getTaskAttemptID().toString(); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java index 312faf35925..63cbef73c82 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java @@ -79,7 +79,17 @@ public class TokenCache { } obtainTokensForNamenodesInternal(credentials, ps, conf); } - + + /** + * Remove jobtoken referrals which don't make sense in the context + * of the task execution. + * + * @param conf + */ + public static void cleanUpTokenReferral(Configuration conf) { + conf.unset(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY); + } + static void obtainTokensForNamenodesInternal(Credentials credentials, Path[] ps, Configuration conf) throws IOException { for(Path p: ps) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java index 607fb347e88..ad6f90d12eb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java @@ -338,7 +338,7 @@ public class CLI extends Configured implements Tool { LogParams logParams = cluster.getLogParams(jobID, taskAttemptID); LogDumper logDumper = new LogDumper(); logDumper.setConf(getConf()); - logDumper.dumpAContainersLogs(logParams.getApplicationId(), + exitCode = logDumper.dumpAContainersLogs(logParams.getApplicationId(), logParams.getContainerId(), logParams.getNodeId(), logParams.getOwner()); } catch (IOException e) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java index 2a2238587f5..e8a67cd848f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java @@ -105,10 +105,9 @@ public class TestFileOutputCommitter extends TestCase { // do commit committer.commitTask(tContext); - Path jobTempDir1 = new Path(outDir, - FileOutputCommitter.getJobAttemptBaseDirName( - conf.getInt(MRConstants.APPLICATION_ATTEMPT_ID, 0))); - assertTrue((new File(jobTempDir1.toString()).exists())); + Path jobTempDir1 = committer.getCommittedTaskPath(tContext); + File jtd1 = new File(jobTempDir1.toUri().getPath()); + assertTrue(jtd1.exists()); validateContent(jobTempDir1); //now while running the second app attempt, @@ -119,14 +118,12 @@ public class TestFileOutputCommitter extends TestCase { JobContext jContext2 = new JobContextImpl(conf2, taskID.getJobID()); TaskAttemptContext tContext2 = new TaskAttemptContextImpl(conf2, taskID); FileOutputCommitter committer2 = new FileOutputCommitter(); - committer.setupJob(jContext2); - Path jobTempDir2 = new Path(outDir, - FileOutputCommitter.getJobAttemptBaseDirName( - conf2.getInt(MRConstants.APPLICATION_ATTEMPT_ID, 0))); - assertTrue((new File(jobTempDir2.toString()).exists())); + committer2.setupJob(jContext2); + Path jobTempDir2 = committer2.getCommittedTaskPath(tContext2); - tContext2.getConfiguration().setInt(MRConstants.APPLICATION_ATTEMPT_ID, 2); committer2.recoverTask(tContext2); + File jtd2 = new File(jobTempDir2.toUri().getPath()); + assertTrue(jtd2.exists()); validateContent(jobTempDir2); committer2.commitJob(jContext2); @@ -135,7 +132,8 @@ public class TestFileOutputCommitter extends TestCase { } private void validateContent(Path dir) throws IOException { - File expectedFile = new File(new Path(dir, partFile).toString()); + File fdir = new File(dir.toUri().getPath()); + File expectedFile = new File(fdir, partFile); StringBuffer expectedOutput = new StringBuffer(); expectedOutput.append(key1).append('\t').append(val1).append("\n"); expectedOutput.append(val1).append("\n"); @@ -244,21 +242,17 @@ public class TestFileOutputCommitter extends TestCase { // do abort committer.abortTask(tContext); - FileSystem outputFileSystem = outDir.getFileSystem(conf); - Path workPath = new Path(outDir, - committer.getTaskAttemptBaseDirName(tContext)) - .makeQualified(outputFileSystem); - File expectedFile = new File(new Path(workPath, partFile) - .toString()); + File out = new File(outDir.toUri().getPath()); + Path workPath = committer.getWorkPath(tContext, outDir); + File wp = new File(workPath.toUri().getPath()); + File expectedFile = new File(wp, partFile); assertFalse("task temp dir still exists", expectedFile.exists()); committer.abortJob(jContext, JobStatus.State.FAILED); - expectedFile = new File(new Path(outDir, FileOutputCommitter.TEMP_DIR_NAME) - .toString()); + expectedFile = new File(out, FileOutputCommitter.TEMP_DIR_NAME); assertFalse("job temp dir still exists", expectedFile.exists()); - assertEquals("Output directory not empty", 0, new File(outDir.toString()) - .listFiles().length); - FileUtil.fullyDelete(new File(outDir.toString())); + assertEquals("Output directory not empty", 0, out.listFiles().length); + FileUtil.fullyDelete(out); } public static class FakeFileSystem extends RawLocalFileSystem { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java index 6708d0443c7..d20480ebba9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java @@ -60,6 +60,22 @@ public class TestFileOutputCommitter extends TestCase { private Text val2 = new Text("val2"); + private static void cleanup() throws IOException { + Configuration conf = new Configuration(); + FileSystem fs = outDir.getFileSystem(conf); + fs.delete(outDir, true); + } + + @Override + public void setUp() throws IOException { + cleanup(); + } + + @Override + public void tearDown() throws IOException { + cleanup(); + } + private void writeOutput(RecordWriter theRecordWriter, TaskAttemptContext context) throws IOException, InterruptedException { NullWritable nullWritable = NullWritable.get(); @@ -114,11 +130,10 @@ public class TestFileOutputCommitter extends TestCase { // do commit committer.commitTask(tContext); - Path jobTempDir1 = new Path(outDir, - FileOutputCommitter.getJobAttemptBaseDirName( - conf.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0))); - assertTrue((new File(jobTempDir1.toString()).exists())); - validateContent(jobTempDir1); + Path jobTempDir1 = committer.getCommittedTaskPath(tContext); + File jtd = new File(jobTempDir1.toUri().getPath()); + assertTrue(jtd.exists()); + validateContent(jtd); //now while running the second app attempt, //recover the task output from first attempt @@ -128,15 +143,13 @@ public class TestFileOutputCommitter extends TestCase { JobContext jContext2 = new JobContextImpl(conf2, taskID.getJobID()); TaskAttemptContext tContext2 = new TaskAttemptContextImpl(conf2, taskID); FileOutputCommitter committer2 = new FileOutputCommitter(outDir, tContext2); - committer.setupJob(tContext2); - Path jobTempDir2 = new Path(outDir, - FileOutputCommitter.getJobAttemptBaseDirName( - conf2.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0))); - assertTrue((new File(jobTempDir2.toString()).exists())); + committer2.setupJob(tContext2); + Path jobTempDir2 = committer2.getCommittedTaskPath(tContext2); + File jtd2 = new File(jobTempDir2.toUri().getPath()); - tContext2.getConfiguration().setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 2); committer2.recoverTask(tContext2); - validateContent(jobTempDir2); + assertTrue(jtd2.exists()); + validateContent(jtd2); committer2.commitJob(jContext2); validateContent(outDir); @@ -144,7 +157,12 @@ public class TestFileOutputCommitter extends TestCase { } private void validateContent(Path dir) throws IOException { - File expectedFile = new File(new Path(dir, partFile).toString()); + validateContent(new File(dir.toUri().getPath())); + } + + private void validateContent(File dir) throws IOException { + File expectedFile = new File(dir, partFile); + assertTrue("Could not find "+expectedFile, expectedFile.exists()); StringBuffer expectedOutput = new StringBuffer(); expectedOutput.append(key1).append('\t').append(val1).append("\n"); expectedOutput.append(val1).append("\n"); @@ -259,7 +277,7 @@ public class TestFileOutputCommitter extends TestCase { assertFalse("task temp dir still exists", expectedFile.exists()); committer.abortJob(jContext, JobStatus.State.FAILED); - expectedFile = new File(new Path(outDir, FileOutputCommitter.TEMP_DIR_NAME) + expectedFile = new File(new Path(outDir, FileOutputCommitter.PENDING_DIR_NAME) .toString()); assertFalse("job temp dir still exists", expectedFile.exists()); assertEquals("Output directory not empty", 0, new File(outDir.toString()) @@ -315,12 +333,10 @@ public class TestFileOutputCommitter extends TestCase { assertNotNull(th); assertTrue(th instanceof IOException); assertTrue(th.getMessage().contains("fake delete failed")); - File jobTmpDir = new File(new Path(outDir, - FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + - conf.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0) + - Path.SEPARATOR + - FileOutputCommitter.TEMP_DIR_NAME).toString()); - File taskTmpDir = new File(jobTmpDir, "_" + taskID); + Path jtd = committer.getJobAttemptPath(jContext); + File jobTmpDir = new File(jtd.toUri().getPath()); + Path ttd = committer.getTaskAttemptPath(tContext); + File taskTmpDir = new File(ttd.toUri().getPath()); File expectedFile = new File(taskTmpDir, partFile); assertTrue(expectedFile + " does not exists", expectedFile.exists()); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java index 8c084fe0ba6..5efc33ddc4e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java @@ -19,6 +19,8 @@ package org.apache.hadoop.mapreduce.security; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -33,6 +35,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.Master; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -158,4 +161,13 @@ public class TestTokenCache { return mockFs; } + + @Test + public void testCleanUpTokenReferral() throws Exception { + Configuration conf = new Configuration(); + conf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, "foo"); + TokenCache.cleanUpTokenReferral(conf); + assertNull(conf.get(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY)); + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java index c3afb013c5e..d0465d37aef 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java @@ -249,8 +249,9 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job } if (historyFileAbsolute != null) { + JobHistoryParser parser = null; try { - JobHistoryParser parser = + parser = new JobHistoryParser(historyFileAbsolute.getFileSystem(conf), historyFileAbsolute); jobInfo = parser.parse(); @@ -258,6 +259,12 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job throw new YarnException("Could not load history file " + historyFileAbsolute, e); } + IOException parseException = parser.getParseException(); + if (parseException != null) { + throw new YarnException( + "Could not parse history file " + historyFileAbsolute, + parseException); + } } else { throw new IOException("History file not found"); } @@ -321,9 +328,6 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job @Override public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) { - if (!UserGroupInformation.isSecurityEnabled()) { - return true; - } Map jobACLs = jobInfo.getJobACLs(); AccessControlList jobACL = jobACLs.get(jobOperation); return aclsMgr.checkAccess(callerUGI, jobOperation, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java index dac0808e576..83380ea5f87 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java @@ -152,7 +152,7 @@ public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job { @Override public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) { - return false; + return true; } @Override diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java index 3755eba9466..d737cd23766 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.StringTokenizer; +import java.util.concurrent.atomic.AtomicInteger; import junit.framework.Assert; @@ -37,14 +38,18 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TypeConverter; +import org.apache.hadoop.mapreduce.jobhistory.EventReader; +import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo; +import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.JobState; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; +import org.apache.hadoop.mapreduce.v2.api.records.TaskState; import org.apache.hadoop.mapreduce.v2.app.MRApp; import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Task; @@ -61,6 +66,9 @@ import org.apache.hadoop.yarn.service.Service; import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.RackResolver; import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; public class TestJobHistoryParsing { private static final Log LOG = LogFactory.getLog(TestJobHistoryParsing.class); @@ -76,6 +84,17 @@ public class TestJobHistoryParsing { @Test public void testHistoryParsing() throws Exception { + checkHistoryParsing(2, 1, 2); + } + + @Test + public void testHistoryParsingWithParseErrors() throws Exception { + checkHistoryParsing(3, 0, 2); + } + + private void checkHistoryParsing(final int numMaps, final int numReduces, + final int numSuccessfulMaps) + throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name")); long amStartTimeEst = System.currentTimeMillis(); @@ -83,8 +102,9 @@ public class TestJobHistoryParsing { CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, MyResolver.class, DNSToSwitchMapping.class); RackResolver.init(conf); - MRApp app = new MRAppWithHistory(2, 1, true, this.getClass().getName(), - true); + MRApp app = + new MRAppWithHistory(numMaps, numReduces, true, + this.getClass().getName(), true); app.submit(conf); Job job = app.getContext().getAllJobs().values().iterator().next(); JobId jobId = job.getID(); @@ -117,8 +137,42 @@ public class TestJobHistoryParsing { } JobHistoryParser parser = new JobHistoryParser(in); - JobInfo jobInfo = parser.parse(); - + final EventReader realReader = new EventReader(in); + EventReader reader = Mockito.mock(EventReader.class); + if (numMaps == numSuccessfulMaps) { + reader = realReader; + } else { + final AtomicInteger numFinishedEvents = new AtomicInteger(0); // Hack! + Mockito.when(reader.getNextEvent()).thenAnswer( + new Answer() { + public HistoryEvent answer(InvocationOnMock invocation) + throws IOException { + HistoryEvent event = realReader.getNextEvent(); + if (event instanceof TaskFinishedEvent) { + numFinishedEvents.incrementAndGet(); + } + + if (numFinishedEvents.get() <= numSuccessfulMaps) { + return event; + } else { + throw new IOException("test"); + } + } + } + ); + } + + JobInfo jobInfo = parser.parse(reader); + + long numFinishedMaps = + computeFinishedMaps(jobInfo, numMaps, numSuccessfulMaps); + + if (numFinishedMaps != numMaps) { + Exception parseException = parser.getParseException(); + Assert.assertNotNull("Didn't get expected parse exception", + parseException); + } + Assert.assertEquals("Incorrect username ", System.getProperty("user.name"), jobInfo.getUsername()); Assert.assertEquals("Incorrect jobName ", "test", jobInfo.getJobname()); @@ -126,14 +180,16 @@ public class TestJobHistoryParsing { jobInfo.getJobQueueName()); Assert .assertEquals("incorrect conf path", "test", jobInfo.getJobConfPath()); - Assert.assertEquals("incorrect finishedMap ", 2, jobInfo.getFinishedMaps()); - Assert.assertEquals("incorrect finishedReduces ", 1, + Assert.assertEquals("incorrect finishedMap ", numSuccessfulMaps, + numFinishedMaps); + Assert.assertEquals("incorrect finishedReduces ", numReduces, jobInfo.getFinishedReduces()); Assert.assertEquals("incorrect uberized ", job.isUber(), jobInfo.getUberized()); Map allTasks = jobInfo.getAllTasks(); int totalTasks = allTasks.size(); - Assert.assertEquals("total number of tasks is incorrect ", 3, totalTasks); + Assert.assertEquals("total number of tasks is incorrect ", + (numMaps+numReduces), totalTasks); // Verify aminfo Assert.assertEquals(1, jobInfo.getAMInfos().size()); @@ -172,55 +228,78 @@ public class TestJobHistoryParsing { Assert.assertNotNull("TaskAttemptInfo not found", taskAttemptInfo); Assert.assertEquals("Incorrect shuffle port for task attempt", taskAttempt.getShufflePort(), taskAttemptInfo.getShufflePort()); - Assert.assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname()); - Assert.assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort()); - - // Verify rack-name - Assert.assertEquals("rack-name is incorrect", taskAttemptInfo - .getRackname(), RACK_NAME); + if (numMaps == numSuccessfulMaps) { + Assert.assertEquals(MRApp.NM_HOST, taskAttemptInfo.getHostname()); + Assert.assertEquals(MRApp.NM_PORT, taskAttemptInfo.getPort()); + + // Verify rack-name + Assert.assertEquals("rack-name is incorrect", taskAttemptInfo + .getRackname(), RACK_NAME); + } } } - String summaryFileName = JobHistoryUtils - .getIntermediateSummaryFileName(jobId); - Path summaryFile = new Path(jobhistoryDir, summaryFileName); - String jobSummaryString = jobHistory.getJobSummary(fc, summaryFile); - Assert.assertTrue(jobSummaryString.contains("resourcesPerMap=100")); - Assert.assertTrue(jobSummaryString.contains("resourcesPerReduce=100")); - Assert.assertNotNull(jobSummaryString); + if (numMaps == numSuccessfulMaps) { - Map jobSummaryElements = new HashMap(); - StringTokenizer strToken = new StringTokenizer(jobSummaryString, ","); - while (strToken.hasMoreTokens()) { - String keypair = strToken.nextToken(); - jobSummaryElements.put(keypair.split("=")[0], keypair.split("=")[1]); + String summaryFileName = JobHistoryUtils + .getIntermediateSummaryFileName(jobId); + Path summaryFile = new Path(jobhistoryDir, summaryFileName); + String jobSummaryString = jobHistory.getJobSummary(fc, summaryFile); + Assert.assertTrue(jobSummaryString.contains("resourcesPerMap=100")); + Assert.assertTrue(jobSummaryString.contains("resourcesPerReduce=100")); + Assert.assertNotNull(jobSummaryString); + Map jobSummaryElements = new HashMap(); + StringTokenizer strToken = new StringTokenizer(jobSummaryString, ","); + while (strToken.hasMoreTokens()) { + String keypair = strToken.nextToken(); + jobSummaryElements.put(keypair.split("=")[0], keypair.split("=")[1]); + + } + + Assert.assertEquals("JobId does not match", jobId.toString(), + jobSummaryElements.get("jobId")); + Assert.assertTrue("submitTime should not be 0", + Long.parseLong(jobSummaryElements.get("submitTime")) != 0); + Assert.assertTrue("launchTime should not be 0", + Long.parseLong(jobSummaryElements.get("launchTime")) != 0); + Assert.assertTrue("firstMapTaskLaunchTime should not be 0", + Long.parseLong(jobSummaryElements.get("firstMapTaskLaunchTime")) != 0); + Assert + .assertTrue( + "firstReduceTaskLaunchTime should not be 0", + Long.parseLong(jobSummaryElements.get("firstReduceTaskLaunchTime")) != 0); + Assert.assertTrue("finishTime should not be 0", + Long.parseLong(jobSummaryElements.get("finishTime")) != 0); + Assert.assertEquals("Mismatch in num map slots", numSuccessfulMaps, + Integer.parseInt(jobSummaryElements.get("numMaps"))); + Assert.assertEquals("Mismatch in num reduce slots", numReduces, + Integer.parseInt(jobSummaryElements.get("numReduces"))); + Assert.assertEquals("User does not match", System.getProperty("user.name"), + jobSummaryElements.get("user")); + Assert.assertEquals("Queue does not match", "default", + jobSummaryElements.get("queue")); + Assert.assertEquals("Status does not match", "SUCCEEDED", + jobSummaryElements.get("status")); } - - Assert.assertEquals("JobId does not match", jobId.toString(), - jobSummaryElements.get("jobId")); - Assert.assertTrue("submitTime should not be 0", - Long.parseLong(jobSummaryElements.get("submitTime")) != 0); - Assert.assertTrue("launchTime should not be 0", - Long.parseLong(jobSummaryElements.get("launchTime")) != 0); - Assert.assertTrue("firstMapTaskLaunchTime should not be 0", - Long.parseLong(jobSummaryElements.get("firstMapTaskLaunchTime")) != 0); - Assert - .assertTrue( - "firstReduceTaskLaunchTime should not be 0", - Long.parseLong(jobSummaryElements.get("firstReduceTaskLaunchTime")) != 0); - Assert.assertTrue("finishTime should not be 0", - Long.parseLong(jobSummaryElements.get("finishTime")) != 0); - Assert.assertEquals("Mismatch in num map slots", 2, - Integer.parseInt(jobSummaryElements.get("numMaps"))); - Assert.assertEquals("Mismatch in num reduce slots", 1, - Integer.parseInt(jobSummaryElements.get("numReduces"))); - Assert.assertEquals("User does not match", System.getProperty("user.name"), - jobSummaryElements.get("user")); - Assert.assertEquals("Queue does not match", "default", - jobSummaryElements.get("queue")); - Assert.assertEquals("Status does not match", "SUCCEEDED", - jobSummaryElements.get("status")); + } + + // Computes finished maps similar to RecoveryService... + private long computeFinishedMaps(JobInfo jobInfo, + int numMaps, int numSuccessfulMaps) { + if (numMaps == numSuccessfulMaps) { + return jobInfo.getFinishedMaps(); + } + + long numFinishedMaps = 0; + Map taskInfos = + jobInfo.getAllTasks(); + for (TaskInfo taskInfo : taskInfos.values()) { + if (TaskState.SUCCEEDED.toString().equals(taskInfo.getTaskStatus())) { + ++numFinishedMaps; + } + } + return numFinishedMaps; } @Test @@ -264,6 +343,9 @@ public class TestJobHistoryParsing { JobHistoryParser parser = new JobHistoryParser(in); JobInfo jobInfo = parser.parse(); + Exception parseException = parser.getParseException(); + Assert.assertNull("Caught an expected exception " + parseException, + parseException); int noOffailedAttempts = 0; Map allTasks = jobInfo.getAllTasks(); for (Task task : job.getTasks().values()) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java index c645cd2592f..a41a0704458 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputCommitter.java @@ -74,7 +74,7 @@ public class TestFileOutputCommitter extends TestCase { TaskAttemptContext tContext = new TaskAttemptContextImpl(job, taskID); FileOutputCommitter committer = new FileOutputCommitter(); FileOutputFormat.setWorkOutputPath(job, - committer.getTempTaskOutputPath(tContext)); + committer.getTaskAttemptPath(tContext)); committer.setupJob(jContext); committer.setupTask(tContext); @@ -115,7 +115,7 @@ public class TestFileOutputCommitter extends TestCase { TaskAttemptContext tContext = new TaskAttemptContextImpl(job, taskID); FileOutputCommitter committer = new FileOutputCommitter(); FileOutputFormat.setWorkOutputPath(job, committer - .getTempTaskOutputPath(tContext)); + .getTaskAttemptPath(tContext)); // do setup committer.setupJob(jContext); @@ -134,13 +134,13 @@ public class TestFileOutputCommitter extends TestCase { // do abort committer.abortTask(tContext); File expectedFile = new File(new Path(committer - .getTempTaskOutputPath(tContext), file).toString()); + .getTaskAttemptPath(tContext), file).toString()); assertFalse("task temp dir still exists", expectedFile.exists()); committer.abortJob(jContext, JobStatus.State.FAILED); expectedFile = new File(new Path(outDir, FileOutputCommitter.TEMP_DIR_NAME) .toString()); - assertFalse("job temp dir still exists", expectedFile.exists()); + assertFalse("job temp dir "+expectedFile+" still exists", expectedFile.exists()); assertEquals("Output directory not empty", 0, new File(outDir.toString()) .listFiles().length); FileUtil.fullyDelete(new File(outDir.toString())); @@ -170,16 +170,15 @@ public class TestFileOutputCommitter extends TestCase { TaskAttemptContext tContext = new TaskAttemptContextImpl(job, taskID); FileOutputCommitter committer = new FileOutputCommitter(); FileOutputFormat.setWorkOutputPath(job, committer - .getTempTaskOutputPath(tContext)); + .getTaskAttemptPath(tContext)); // do setup committer.setupJob(jContext); committer.setupTask(tContext); String file = "test.txt"; - String taskBaseDirName = committer.getTaskAttemptBaseDirName(tContext); - File jobTmpDir = new File(outDir.toString(), committer.getJobAttemptBaseDirName(jContext)); - File taskTmpDir = new File(outDir.toString(), taskBaseDirName); + File jobTmpDir = new File(committer.getJobAttemptPath(jContext).toUri().getPath()); + File taskTmpDir = new File(committer.getTaskAttemptPath(tContext).toUri().getPath()); File expectedFile = new File(taskTmpDir, file); // A reporter that does nothing diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java index 4d1224ce0aa..ea0f88c7a65 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java @@ -34,7 +34,7 @@ public class TestTaskCommit extends HadoopTestCase { static class CommitterWithCommitFail extends FileOutputCommitter { public void commitTask(TaskAttemptContext context) throws IOException { - Path taskOutputPath = getTempTaskOutputPath(context); + Path taskOutputPath = getTaskAttemptPath(context); TaskAttemptID attemptId = context.getTaskAttemptID(); JobConf job = context.getJobConf(); if (taskOutputPath != null) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java index 5001400bbe1..c7d4e51bd2c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java @@ -70,6 +70,22 @@ public class TestFileOutputCommitter extends TestCase { } } + private static void cleanup() throws IOException { + Configuration conf = new Configuration(); + FileSystem fs = outDir.getFileSystem(conf); + fs.delete(outDir, true); + } + + @Override + public void setUp() throws IOException { + cleanup(); + } + + @Override + public void tearDown() throws IOException { + cleanup(); + } + @SuppressWarnings("unchecked") public void testCommitter() throws Exception { Job job = Job.getInstance(); @@ -133,7 +149,7 @@ public class TestFileOutputCommitter extends TestCase { assertFalse("task temp dir still exists", expectedFile.exists()); committer.abortJob(jContext, JobStatus.State.FAILED); - expectedFile = new File(new Path(outDir, FileOutputCommitter.TEMP_DIR_NAME) + expectedFile = new File(new Path(outDir, FileOutputCommitter.PENDING_DIR_NAME) .toString()); assertFalse("job temp dir still exists", expectedFile.exists()); assertEquals("Output directory not empty", 0, new File(outDir.toString()) @@ -188,9 +204,9 @@ public class TestFileOutputCommitter extends TestCase { assertNotNull(th); assertTrue(th instanceof IOException); assertTrue(th.getMessage().contains("fake delete failed")); - String taskBaseDirName = committer.getTaskAttemptBaseDirName(tContext); - File jobTmpDir = new File(outDir.toString(), committer.getJobAttemptBaseDirName(jContext)); - File taskTmpDir = new File(outDir.toString(), taskBaseDirName); + //Path taskBaseDirName = committer.getTaskAttemptBaseDirName(tContext); + File jobTmpDir = new File(committer.getJobAttemptPath(jContext).toUri().getPath()); + File taskTmpDir = new File(committer.getTaskAttemptPath(tContext).toUri().getPath()); File expectedFile = new File(taskTmpDir, partFile); assertTrue(expectedFile + " does not exists", expectedFile.exists()); diff --git a/hadoop-mapreduce-project/hadoop-yarn/bin/yarn-daemon.sh b/hadoop-mapreduce-project/hadoop-yarn/bin/yarn-daemon.sh index 6e41f791c3e..1fa43d8b1b2 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/bin/yarn-daemon.sh +++ b/hadoop-mapreduce-project/hadoop-yarn/bin/yarn-daemon.sh @@ -78,8 +78,11 @@ fi if [ "$YARN_LOG_DIR" = "" ]; then export YARN_LOG_DIR="$YARN_HOME/logs" fi -mkdir -p "$YARN_LOG_DIR" -chown $YARN_IDENT_STRING $YARN_LOG_DIR + +if [ ! -w "$YARN_LOG_DIR" ] ; then + mkdir -p "$YARN_LOG_DIR" + chown $YARN_IDENT_STRING $YARN_LOG_DIR +fi if [ "$YARN_PID_DIR" = "" ]; then YARN_PID_DIR=/tmp @@ -101,7 +104,7 @@ case $startStop in (start) - mkdir -p "$YARN_PID_DIR" + [ -w "$YARN_PID_DIR" ] || mkdir -p "$YARN_PID_DIR" if [ -f $pid ]; then if kill -0 `cat $pid` > /dev/null 2>&1; then diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java index 20206d6f941..c81b772e785 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogDumper.java @@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.logaggregation; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; +import java.io.FileNotFoundException; import java.io.IOException; import org.apache.commons.cli.CommandLine; @@ -103,14 +104,15 @@ public class LogDumper extends Configured implements Tool { if (appOwner == null || appOwner.isEmpty()) { appOwner = UserGroupInformation.getCurrentUser().getShortUserName(); } + int resultCode = 0; if (containerIdStr == null && nodeAddress == null) { - dumpAllContainersLogs(appId, appOwner, out); + resultCode = dumpAllContainersLogs(appId, appOwner, out); } else if ((containerIdStr == null && nodeAddress != null) || (containerIdStr != null && nodeAddress == null)) { System.out.println("ContainerId or NodeAddress cannot be null!"); HelpFormatter formatter = new HelpFormatter(); formatter.printHelp("general options are: ", opts); - return -1; + resultCode = -1; } else { Path remoteRootLogDir = new Path(getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, @@ -122,27 +124,33 @@ public class LogDumper extends Configured implements Tool { appId, appOwner, ConverterUtils.toNodeId(nodeAddress), - getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, - YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX))); - return dumpAContainerLogs(containerIdStr, reader, out); + LogAggregationUtils.getRemoteNodeLogDirSuffix(getConf()))); + resultCode = dumpAContainerLogs(containerIdStr, reader, out); } - return 0; + return resultCode; } - public void dumpAContainersLogs(String appId, String containerId, + public int dumpAContainersLogs(String appId, String containerId, String nodeId, String jobOwner) throws IOException { Path remoteRootLogDir = new Path(getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); String suffix = LogAggregationUtils.getRemoteNodeLogDirSuffix(getConf()); - AggregatedLogFormat.LogReader reader = - new AggregatedLogFormat.LogReader(getConf(), - LogAggregationUtils.getRemoteNodeLogFileForApp(remoteRootLogDir, - ConverterUtils.toApplicationId(appId), jobOwner, - ConverterUtils.toNodeId(nodeId), suffix)); + Path logPath = LogAggregationUtils.getRemoteNodeLogFileForApp( + remoteRootLogDir, ConverterUtils.toApplicationId(appId), jobOwner, + ConverterUtils.toNodeId(nodeId), suffix); + AggregatedLogFormat.LogReader reader; + try { + reader = new AggregatedLogFormat.LogReader(getConf(), logPath); + } catch (FileNotFoundException fnfe) { + System.out.println("Logs not available at " + logPath.toString()); + System.out.println( + "Log aggregation has not completed or is not enabled."); + return -1; + } DataOutputStream out = new DataOutputStream(System.out); - dumpAContainerLogs(containerId, reader, out); + return dumpAContainerLogs(containerId, reader, out); } private int dumpAContainerLogs(String containerIdStr, @@ -174,21 +182,28 @@ public class LogDumper extends Configured implements Tool { return 0; } - private void dumpAllContainersLogs(ApplicationId appId, String appOwner, + private int dumpAllContainersLogs(ApplicationId appId, String appOwner, DataOutputStream out) throws IOException { Path remoteRootLogDir = new Path(getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR)); String user = appOwner; String logDirSuffix = - getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, - YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX); + LogAggregationUtils.getRemoteNodeLogDirSuffix(getConf()); //TODO Change this to get a list of files from the LAS. Path remoteAppLogDir = LogAggregationUtils.getRemoteAppLogDir(remoteRootLogDir, appId, user, logDirSuffix); - RemoteIterator nodeFiles = - FileContext.getFileContext().listStatus(remoteAppLogDir); + RemoteIterator nodeFiles; + try { + nodeFiles = FileContext.getFileContext().listStatus(remoteAppLogDir); + } catch (FileNotFoundException fnf) { + System.out.println("Logs not available at " + + remoteAppLogDir.toString()); + System.out.println( + "Log aggregation has not completed or is not enabled."); + return -1; + } while (nodeFiles.hasNext()) { FileStatus thisNodeFile = nodeFiles.next(); AggregatedLogFormat.LogReader reader = @@ -217,12 +232,14 @@ public class LogDumper extends Configured implements Tool { reader.close(); } } + return 0; } public static void main(String[] args) throws Exception { Configuration conf = new YarnConfiguration(); LogDumper logDumper = new LogDumper(); logDumper.setConf(conf); - logDumper.run(args); + int exitCode = logDumper.run(args); + System.exit(exitCode); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestLogDumper.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestLogDumper.java new file mode 100644 index 00000000000..527a551f2d6 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestLogDumper.java @@ -0,0 +1,47 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.hadoop.yarn.logaggregation; + +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.junit.Test; + +public class TestLogDumper { + @Test + public void testFailResultCodes() throws Exception { + Configuration conf = new YarnConfiguration(); + conf.setClass("fs.file.impl", LocalFileSystem.class, FileSystem.class); + LogDumper dumper = new LogDumper(); + dumper.setConf(conf); + + // verify dumping a non-existent application's logs returns a failure code + int exitCode = dumper.run( new String[] { + "-applicationId", "application_0_0" } ); + assertTrue("Should return an error code", exitCode != 0); + + // verify dumping a non-existent container log is a failure code + exitCode = dumper.dumpAContainersLogs("application_0_0", "container_0_0", + "nonexistentnode:1234", "nobody"); + assertTrue("Should return an error code", exitCode != 0); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java index e9e1b2fb0b0..e187fbdbdec 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java @@ -38,7 +38,7 @@ public class ClusterMetrics { private static AtomicBoolean isInitialized = new AtomicBoolean(false); - @Metric("# of active NMs") MutableGaugeInt numNMs; + @Metric("# of active NMs") MutableGaugeInt numActiveNMs; @Metric("# of decommissioned NMs") MutableGaugeInt numDecommissionedNMs; @Metric("# of lost NMs") MutableGaugeInt numLostNMs; @Metric("# of unhealthy NMs") MutableGaugeInt numUnhealthyNMs; @@ -74,7 +74,7 @@ public class ClusterMetrics { //Active Nodemanagers public int getNumActiveNMs() { - return numNMs.value(); + return numActiveNMs.value(); } //Decommisioned NMs @@ -128,17 +128,12 @@ public class ClusterMetrics { public void decrNumRebootedNMs() { numRebootedNMs.decr(); } - - public void removeNode(RMNodeEventType nodeEventType) { - numNMs.decr(); - switch(nodeEventType){ - case DECOMMISSION: incrDecommisionedNMs(); break; - case EXPIRE: incrNumLostNMs();break; - case REBOOTING: incrNumRebootedNMs();break; - } + + public void incrNumActiveNodes() { + numActiveNMs.incr(); } - - public void addNode() { - numNMs.incr(); + + public void decrNumActiveNodes() { + numActiveNMs.decr(); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java index 428246d485d..4f41bfde8a0 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java @@ -111,4 +111,12 @@ public class Resources { public static boolean greaterThanOrEqual(Resource lhs, Resource rhs) { return lhs.getMemory() >= rhs.getMemory(); } + + public static Resource min(Resource lhs, Resource rhs) { + return (lhs.getMemory() < rhs.getMemory()) ? lhs : rhs; + } + + public static Resource max(Resource lhs, Resource rhs) { + return (lhs.getMemory() > rhs.getMemory()) ? lhs : rhs; + } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java index 4e79540733f..9b8892a6dcd 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java @@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.util.BuilderUtils.ContainerIdComparator; */ @Private @Unstable +@SuppressWarnings("unchecked") public class RMNodeImpl implements RMNode, EventHandler { private static final Log LOG = LogFactory.getLog(RMNodeImpl.class); @@ -116,11 +117,14 @@ public class RMNodeImpl implements RMNode, EventHandler { EnumSet.of(RMNodeState.RUNNING, RMNodeState.UNHEALTHY), RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenHealthyTransition()) .addTransition(RMNodeState.RUNNING, RMNodeState.DECOMMISSIONED, - RMNodeEventType.DECOMMISSION, new RemoveNodeTransition()) + RMNodeEventType.DECOMMISSION, + new DeactivateNodeTransition(RMNodeState.DECOMMISSIONED)) .addTransition(RMNodeState.RUNNING, RMNodeState.LOST, - RMNodeEventType.EXPIRE, new RemoveNodeTransition()) + RMNodeEventType.EXPIRE, + new DeactivateNodeTransition(RMNodeState.LOST)) .addTransition(RMNodeState.RUNNING, RMNodeState.REBOOTED, - RMNodeEventType.REBOOTING, new RemoveNodeTransition()) + RMNodeEventType.REBOOTING, + new DeactivateNodeTransition(RMNodeState.REBOOTED)) .addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING, RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition()) .addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING, @@ -304,26 +308,50 @@ public class RMNodeImpl implements RMNode, EventHandler { writeLock.unlock(); } } - + + private void updateMetricsForRejoinedNode(RMNodeState previousNodeState) { + ClusterMetrics metrics = ClusterMetrics.getMetrics(); + metrics.incrNumActiveNodes(); + + switch (previousNodeState) { + case LOST: + metrics.decrNumLostNMs(); + break; + case REBOOTED: + metrics.decrNumRebootedNMs(); + break; + case DECOMMISSIONED: + metrics.decrDecommisionedNMs(); + break; + case UNHEALTHY: + metrics.decrNumUnhealthyNMs(); + break; + } + } + + private void updateMetricsForDeactivatedNode(RMNodeState finalState) { + ClusterMetrics metrics = ClusterMetrics.getMetrics(); + metrics.decrNumActiveNodes(); + + switch (finalState) { + case DECOMMISSIONED: + metrics.incrDecommisionedNMs(); + break; + case LOST: + metrics.incrNumLostNMs(); + break; + case REBOOTED: + metrics.incrNumRebootedNMs(); + break; + case UNHEALTHY: + metrics.incrNumUnhealthyNMs(); + break; + } + } + public static class AddNodeTransition implements SingleArcTransition { - - private void updateMetrics(RMNodeState nodeState) { - ClusterMetrics metrics = ClusterMetrics.getMetrics(); - switch (nodeState) { - case LOST: - metrics.decrNumLostNMs(); - break; - case REBOOTED: - metrics.decrNumRebootedNMs(); - break; - case DECOMMISSIONED: - metrics.decrDecommisionedNMs(); - break; - } - } - @SuppressWarnings("unchecked") @Override public void transition(RMNodeImpl rmNode, RMNodeEvent event) { // Inform the scheduler @@ -333,12 +361,14 @@ public class RMNodeImpl implements RMNode, EventHandler { String host = rmNode.nodeId.getHost(); if (rmNode.context.getInactiveRMNodes().containsKey(host)) { - RMNode node = rmNode.context.getInactiveRMNodes().get(host); + // Old node rejoining + RMNode previouRMNode = rmNode.context.getInactiveRMNodes().get(host); rmNode.context.getInactiveRMNodes().remove(host); - updateMetrics(node.getState()); + rmNode.updateMetricsForRejoinedNode(previouRMNode.getState()); + } else { + // Increment activeNodes explicitly because this is a new node. + ClusterMetrics.getMetrics().incrNumActiveNodes(); } - - ClusterMetrics.getMetrics().addNode(); } } @@ -362,28 +392,33 @@ public class RMNodeImpl implements RMNode, EventHandler { } } - public static class RemoveNodeTransition + public static class DeactivateNodeTransition implements SingleArcTransition { - @SuppressWarnings("unchecked") + private final RMNodeState finalState; + public DeactivateNodeTransition(RMNodeState finalState) { + this.finalState = finalState; + } + @Override public void transition(RMNodeImpl rmNode, RMNodeEvent event) { // Inform the scheduler rmNode.context.getDispatcher().getEventHandler().handle( new NodeRemovedSchedulerEvent(rmNode)); - // Remove the node from the system. + // Deactivate the node rmNode.context.getRMNodes().remove(rmNode.nodeId); - LOG.info("Removed Node " + rmNode.nodeId); + LOG.info("Deactivating Node " + rmNode.nodeId + " as it is now " + + finalState); rmNode.context.getInactiveRMNodes().put(rmNode.nodeId.getHost(), rmNode); - //Update the metrics - ClusterMetrics.getMetrics().removeNode(event.getType()); + + //Update the metrics + rmNode.updateMetricsForDeactivatedNode(finalState); } } public static class StatusUpdateWhenHealthyTransition implements MultipleArcTransition { - @SuppressWarnings("unchecked") @Override public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { @@ -399,7 +434,8 @@ public class RMNodeImpl implements RMNode, EventHandler { // Inform the scheduler rmNode.context.getDispatcher().getEventHandler().handle( new NodeRemovedSchedulerEvent(rmNode)); - ClusterMetrics.getMetrics().incrNumUnhealthyNMs(); + // Update metrics + rmNode.updateMetricsForDeactivatedNode(RMNodeState.UNHEALTHY); return RMNodeState.UNHEALTHY; } @@ -458,11 +494,9 @@ public class RMNodeImpl implements RMNode, EventHandler { } } - public static class StatusUpdateWhenUnHealthyTransition - implements + public static class StatusUpdateWhenUnHealthyTransition implements MultipleArcTransition { - @SuppressWarnings("unchecked") @Override public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { RMNodeStatusEvent statusEvent = (RMNodeStatusEvent) event; @@ -474,7 +508,8 @@ public class RMNodeImpl implements RMNode, EventHandler { if (remoteNodeHealthStatus.getIsNodeHealthy()) { rmNode.context.getDispatcher().getEventHandler().handle( new NodeAddedSchedulerEvent(rmNode)); - ClusterMetrics.getMetrics().decrNumUnhealthyNMs(); + // Update metrics + rmNode.updateMetricsForRejoinedNode(RMNodeState.UNHEALTHY); return RMNodeState.RUNNING; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java index 2040505be67..0489ab287eb 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java @@ -162,6 +162,13 @@ public class AppSchedulingInfo { asks.put(hostName, request); if (updatePendingResources) { + + // Similarly, deactivate application? + if (request.getNumContainers() <= 0) { + LOG.info("checking for deactivate... "); + checkForDeactivation(); + } + int lastRequestContainers = lastRequest != null ? lastRequest .getNumContainers() : 0; Resource lastRequestCapability = lastRequest != null ? lastRequest @@ -308,19 +315,24 @@ public class AppSchedulingInfo { // Do we have any outstanding requests? // If there is nothing, we need to deactivate this application if (numOffSwitchContainers == 0) { - boolean deactivate = true; - for (Priority priority : getPriorities()) { - ResourceRequest request = getResourceRequest(priority, RMNodeImpl.ANY); - if (request.getNumContainers() > 0) { - deactivate = false; - break; - } - } - if (deactivate) { - activeUsersManager.deactivateApplication(user, applicationId); - } + checkForDeactivation(); } } + + synchronized private void checkForDeactivation() { + boolean deactivate = true; + for (Priority priority : getPriorities()) { + ResourceRequest request = getResourceRequest(priority, RMNodeImpl.ANY); + if (request.getNumContainers() > 0) { + deactivate = false; + break; + } + } + if (deactivate) { + activeUsersManager.deactivateApplication(user, applicationId); + } + } + synchronized private void allocate(Container container) { // Update consumption and track allocations //TODO: fixme sharad diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java index 7d38dcdbe41..89c36ab87aa 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java @@ -46,17 +46,23 @@ class CSQueueUtils { } public static int computeMaxActiveApplications(Resource clusterResource, - float maxAMResourcePercent, float absoluteCapacity) { + Resource minimumAllocation, float maxAMResourcePercent, + float absoluteMaxCapacity) { return Math.max( - (int)((clusterResource.getMemory() / (float)LeafQueue.DEFAULT_AM_RESOURCE) * - maxAMResourcePercent * absoluteCapacity), + (int)Math.ceil( + ((float)clusterResource.getMemory() / + minimumAllocation.getMemory()) * + maxAMResourcePercent * absoluteMaxCapacity), 1); } public static int computeMaxActiveApplicationsPerUser( int maxActiveApplications, int userLimit, float userLimitFactor) { - return (int)(maxActiveApplications * (userLimit / 100.0f) * userLimitFactor); + return Math.max( + (int)Math.ceil( + maxActiveApplications * (userLimit / 100.0f) * userLimitFactor), + 1); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index 08ee09463a0..ecc672c1cb8 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -125,8 +125,6 @@ public class LeafQueue implements CSQueue { private final ActiveUsersManager activeUsersManager; - final static int DEFAULT_AM_RESOURCE = 2 * 1024; - public LeafQueue(CapacitySchedulerContext cs, String queueName, CSQueue parent, Comparator applicationComparator, CSQueue old) { @@ -166,8 +164,9 @@ public class LeafQueue implements CSQueue { this.maxAMResourcePercent = cs.getConfiguration().getMaximumApplicationMasterResourcePercent(); int maxActiveApplications = - CSQueueUtils.computeMaxActiveApplications(cs.getClusterResources(), - maxAMResourcePercent, absoluteCapacity); + CSQueueUtils.computeMaxActiveApplications( + cs.getClusterResources(), this.minimumAllocation, + maxAMResourcePercent, absoluteMaxCapacity); int maxActiveApplicationsPerUser = CSQueueUtils.computeMaxActiveApplicationsPerUser(maxActiveApplications, userLimit, userLimitFactor); @@ -246,30 +245,39 @@ public class LeafQueue implements CSQueue { " [= configuredMaxCapacity ]" + "\n" + "absoluteMaxCapacity = " + absoluteMaxCapacity + " [= 1.0 maximumCapacity undefined, " + - "(parentAbsoluteMaxCapacity * maximumCapacity) / 100 otherwise ]" + "\n" + + "(parentAbsoluteMaxCapacity * maximumCapacity) / 100 otherwise ]" + + "\n" + "userLimit = " + userLimit + " [= configuredUserLimit ]" + "\n" + "userLimitFactor = " + userLimitFactor + " [= configuredUserLimitFactor ]" + "\n" + "maxApplications = " + maxApplications + - " [= (int)(configuredMaximumSystemApplications * absoluteCapacity) ]" + "\n" + + " [= (int)(configuredMaximumSystemApplications * absoluteCapacity) ]" + + "\n" + "maxApplicationsPerUser = " + maxApplicationsPerUser + - " [= (int)(maxApplications * (userLimit / 100.0f) * userLimitFactor) ]" + "\n" + + " [= (int)(maxApplications * (userLimit / 100.0f) * " + + "userLimitFactor) ]" + "\n" + "maxActiveApplications = " + maxActiveApplications + " [= max(" + - "(int)((clusterResourceMemory / (float)DEFAULT_AM_RESOURCE) *" + - "maxAMResourcePercent * absoluteCapacity)," + + "(int)ceil((clusterResourceMemory / minimumAllocation) *" + + "maxAMResourcePercent * absoluteMaxCapacity)," + "1) ]" + "\n" + "maxActiveApplicationsPerUser = " + maxActiveApplicationsPerUser + - " [= (int)(maxActiveApplications * (userLimit / 100.0f) * userLimitFactor) ]" + "\n" + + " [= max(" + + "(int)(maxActiveApplications * (userLimit / 100.0f) * " + + "userLimitFactor)," + + "1) ]" + "\n" + "utilization = " + utilization + - " [= usedResourcesMemory / (clusterResourceMemory * absoluteCapacity)]" + "\n" + + " [= usedResourcesMemory / " + + "(clusterResourceMemory * absoluteCapacity)]" + "\n" + "usedCapacity = " + usedCapacity + - " [= usedResourcesMemory / (clusterResourceMemory * parent.absoluteCapacity)]" + "\n" + + " [= usedResourcesMemory / " + + "(clusterResourceMemory * parent.absoluteCapacity)]" + "\n" + "maxAMResourcePercent = " + maxAMResourcePercent + " [= configuredMaximumAMResourcePercent ]" + "\n" + "minimumAllocationFactor = " + minimumAllocationFactor + - " [= (float)(maximumAllocationMemory - minimumAllocationMemory) / maximumAllocationMemory ]" + "\n" + + " [= (float)(maximumAllocationMemory - minimumAllocationMemory) / " + + "maximumAllocationMemory ]" + "\n" + "numContainers = " + numContainers + " [= currentNumContainers ]" + "\n" + "state = " + state + @@ -606,7 +614,10 @@ public class LeafQueue implements CSQueue { addApplication(application, user); } - metrics.submitApp(userName); + int attemptId = application.getApplicationAttemptId().getAttemptId(); + if (attemptId == 1) { + metrics.submitApp(userName); + } // Inform the parent queue try { @@ -635,7 +646,7 @@ public class LeafQueue implements CSQueue { user.activateApplication(); activeApplications.add(application); i.remove(); - LOG.info("Application " + application.getApplicationId().getId() + + LOG.info("Application " + application.getApplicationId() + " from user: " + application.getUser() + " activated in queue: " + getQueueName()); } @@ -673,10 +684,13 @@ public class LeafQueue implements CSQueue { } public synchronized void removeApplication(SchedulerApp application, User user) { - activeApplications.remove(application); + boolean wasActive = activeApplications.remove(application); + if (!wasActive) { + pendingApplications.remove(application); + } applicationsMap.remove(application.getApplicationAttemptId()); - user.finishApplication(); + user.finishApplication(wasActive); if (user.getTotalApplications() == 0) { users.remove(application.getUser()); } @@ -751,15 +765,15 @@ public class LeafQueue implements CSQueue { continue; } - // Compute & set headroom - // Note: We set the headroom with the highest priority request - // as the target. + // Compute user-limit & set headroom + // Note: We compute both user-limit & headroom with the highest + // priority request as the target. // This works since we never assign lower priority requests // before all higher priority ones are serviced. Resource userLimit = - computeAndSetUserResourceLimit(application, clusterResource, - required); - + computeUserLimitAndSetHeadroom(application, clusterResource, + required); + // Check queue max-capacity limit if (!assignToQueue(clusterResource, required)) { return NULL_ASSIGNMENT; @@ -777,13 +791,13 @@ public class LeafQueue implements CSQueue { CSAssignment assignment = assignContainersOnNode(clusterResource, node, application, priority, null); - - Resource assigned = assignment.getResource(); - + // Did we schedule or reserve a container? + Resource assigned = assignment.getResource(); if (Resources.greaterThan(assigned, Resources.none())) { - // Book-keeping + // Book-keeping + // Note: Update headroom to account for current allocation too... allocateResource(clusterResource, application, assigned); // Reset scheduling opportunities @@ -854,20 +868,50 @@ public class LeafQueue implements CSQueue { } @Lock({LeafQueue.class, SchedulerApp.class}) - private Resource computeAndSetUserResourceLimit(SchedulerApp application, - Resource clusterResource, Resource required) { + private Resource computeUserLimitAndSetHeadroom( + SchedulerApp application, Resource clusterResource, Resource required) { + String user = application.getUser(); - Resource limit = computeUserLimit(application, clusterResource, required); + + /** + * Headroom is min((userLimit, queue-max-cap) - consumed) + */ + + Resource userLimit = // User limit + computeUserLimit(application, clusterResource, required); + + + Resource queueMaxCap = // Queue Max-Capacity + Resources.createResource( + roundDown((int)(absoluteMaxCapacity * clusterResource.getMemory())) + ); + + Resource userConsumed = getUser(user).getConsumedResources(); Resource headroom = - Resources.subtract(limit, getUser(user).getConsumedResources()); + Resources.subtract(Resources.min(userLimit, queueMaxCap), userConsumed); + + if (LOG.isDebugEnabled()) { + LOG.debug("Headroom calculation for user " + user + ": " + + " userLimit=" + userLimit + + " queueMaxCap=" + queueMaxCap + + " consumed=" + userConsumed + + " headroom=" + headroom); + } + application.setHeadroom(headroom); metrics.setAvailableResourcesToUser(user, headroom); - return limit; + + return userLimit; } private int roundUp(int memory) { - return divideAndCeil(memory, minimumAllocation.getMemory()) * - minimumAllocation.getMemory(); + int minMemory = minimumAllocation.getMemory(); + return divideAndCeil(memory, minMemory) * minMemory; + } + + private int roundDown(int memory) { + int minMemory = minimumAllocation.getMemory(); + return (memory / minMemory) * minMemory; } @Lock(NoLock.class) @@ -1288,10 +1332,17 @@ public class LeafQueue implements CSQueue { String userName = application.getUser(); User user = getUser(userName); user.assignContainer(resource); + Resources.subtractFrom(application.getHeadroom(), resource); // headroom metrics.setAvailableResourcesToUser(userName, application.getHeadroom()); - LOG.info(getQueueName() + - " used=" + usedResources + " numContainers=" + numContainers + - " user=" + userName + " user-resources=" + user.getConsumedResources()); + + if (LOG.isDebugEnabled()) { + LOG.info(getQueueName() + + " user=" + userName + + " used=" + usedResources + " numContainers=" + numContainers + + " headroom = " + application.getHeadroom() + + " user-resources=" + user.getConsumedResources() + ); + } } synchronized void releaseResource(Resource clusterResource, @@ -1316,17 +1367,18 @@ public class LeafQueue implements CSQueue { public synchronized void updateClusterResource(Resource clusterResource) { // Update queue properties maxActiveApplications = - CSQueueUtils.computeMaxActiveApplications(clusterResource, maxAMResourcePercent, - absoluteCapacity); + CSQueueUtils.computeMaxActiveApplications( + clusterResource, minimumAllocation, + maxAMResourcePercent, absoluteMaxCapacity); maxActiveApplicationsPerUser = - CSQueueUtils.computeMaxActiveApplicationsPerUser(maxActiveApplications, userLimit, - userLimitFactor); + CSQueueUtils.computeMaxActiveApplicationsPerUser( + maxActiveApplications, userLimit, userLimitFactor); // Update application properties for (SchedulerApp application : activeApplications) { synchronized (application) { - computeAndSetUserResourceLimit( - application, clusterResource, Resources.none()); + computeUserLimitAndSetHeadroom(application, clusterResource, + Resources.none()); } } } @@ -1378,8 +1430,13 @@ public class LeafQueue implements CSQueue { ++activeApplications; } - public synchronized void finishApplication() { - --activeApplications; + public synchronized void finishApplication(boolean wasActive) { + if (wasActive) { + --activeApplications; + } + else { + --pendingApplications; + } } public synchronized void assignContainer(Resource resource) { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index 9f5f2cc8ca1..ecdb7589234 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -298,7 +298,9 @@ public class FifoScheduler implements ResourceScheduler { new SchedulerApp(appAttemptId, user, DEFAULT_QUEUE, activeUsersManager, this.rmContext, null); applications.put(appAttemptId, schedulerApp); - metrics.submitApp(user); + if (appAttemptId.getAttemptId() == 1) { + metrics.submitApp(user); + } LOG.info("Application Submission: " + appAttemptId.getApplicationId() + " from " + user + ", currently active: " + applications.size()); rmContext.getDispatcher().getEventHandler().handle( diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java index cb6b6c5ad09..13d5a743183 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java @@ -100,6 +100,12 @@ class NodesPage extends RmView { if(!stateFilter.equals(state)) { continue; } + } else { + // No filter. User is asking for all nodes. Make sure you skip the + // unhealthy nodes. + if (ni.getState() == RMNodeState.UNHEALTHY) { + continue; + } } NodeInfo info = new NodeInfo(ni, sched); int usedMemory = (int)info.getUsedMemory(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java index 449ba758e86..d745446b46c 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java @@ -166,6 +166,12 @@ public class RMWebServices { if (!(nodeInfo.getState().equalsIgnoreCase(filterState))) { continue; } + } else { + // No filter. User is asking for all nodes. Make sure you skip the + // unhealthy nodes. + if (ni.getState() == RMNodeState.UNHEALTHY) { + continue; + } } if ((healthState != null) && (!healthState.isEmpty())) { LOG.info("heatlh state is : " + healthState); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java index 7ded620043a..05b17a367fe 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java @@ -51,18 +51,23 @@ public class MockNodes { List list = Lists.newArrayList(); for (int i = 0; i < racks; ++i) { for (int j = 0; j < nodesPerRack; ++j) { + if (j == (nodesPerRack - 1)) { + // One unhealthy node per rack. + list.add(nodeInfo(i, perNode, RMNodeState.UNHEALTHY)); + } list.add(newNodeInfo(i, perNode)); } } return list; } - public static List lostNodes(int racks, int nodesPerRack, + public static List deactivatedNodes(int racks, int nodesPerRack, Resource perNode) { List list = Lists.newArrayList(); for (int i = 0; i < racks; ++i) { for (int j = 0; j < nodesPerRack; ++j) { - list.add(lostNodeInfo(i, perNode, RMNodeState.LOST)); + RMNodeState[] allStates = RMNodeState.values(); + list.add(nodeInfo(i, perNode, allStates[j % allStates.length])); } } return list; @@ -198,15 +203,20 @@ public class MockNodes { final String httpAddress = httpAddr; final NodeHealthStatus nodeHealthStatus = recordFactory.newRecordInstance(NodeHealthStatus.class); + if (state != RMNodeState.UNHEALTHY) { + nodeHealthStatus.setIsNodeHealthy(true); + nodeHealthStatus.setHealthReport("HealthyMe"); + } return new MockRMNodeImpl(nodeID, hostName, httpAddress, perNode, rackName, nodeHealthStatus, nid, hostName, state); } - public static RMNode lostNodeInfo(int rack, final Resource perNode, RMNodeState state) { + public static RMNode nodeInfo(int rack, final Resource perNode, + RMNodeState state) { return buildRMNode(rack, perNode, state, "N/A"); } public static RMNode newNodeInfo(int rack, final Resource perNode) { - return buildRMNode(rack, perNode, null, "localhost:0"); + return buildRMNode(rack, perNode, RMNodeState.RUNNING, "localhost:0"); } } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java index 9c5851a5db8..7426d2b273d 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java @@ -153,29 +153,31 @@ public class TestApplicationLimits { queue.getMaximumActiveApplicationsPerUser()); int expectedMaxActiveApps = Math.max(1, - (int)((clusterResource.getMemory() / LeafQueue.DEFAULT_AM_RESOURCE) * + (int)Math.ceil(((float)clusterResource.getMemory() / (1*GB)) * csConf.getMaximumApplicationMasterResourcePercent() * - queue.getAbsoluteCapacity())); + queue.getAbsoluteMaximumCapacity())); assertEquals(expectedMaxActiveApps, queue.getMaximumActiveApplications()); - assertEquals((int)(expectedMaxActiveApps * (queue.getUserLimit() / 100.0f) * - queue.getUserLimitFactor()), - queue.getMaximumActiveApplicationsPerUser()); + assertEquals( + (int)Math.ceil( + expectedMaxActiveApps * (queue.getUserLimit() / 100.0f) * + queue.getUserLimitFactor()), + queue.getMaximumActiveApplicationsPerUser()); // Add some nodes to the cluster & test new limits clusterResource = Resources.createResource(120 * 16 * GB); root.updateClusterResource(clusterResource); expectedMaxActiveApps = Math.max(1, - (int)((clusterResource.getMemory() / LeafQueue.DEFAULT_AM_RESOURCE) * + (int)Math.ceil(((float)clusterResource.getMemory() / (1*GB)) * csConf.getMaximumApplicationMasterResourcePercent() * - queue.getAbsoluteCapacity())); + queue.getAbsoluteMaximumCapacity())); assertEquals(expectedMaxActiveApps, queue.getMaximumActiveApplications()); - assertEquals((int)(expectedMaxActiveApps * (queue.getUserLimit() / 100.0f) * - queue.getUserLimitFactor()), - queue.getMaximumActiveApplicationsPerUser()); - + assertEquals( + (int)Math.ceil(expectedMaxActiveApps * + (queue.getUserLimit() / 100.0f) * queue.getUserLimitFactor()), + queue.getMaximumActiveApplicationsPerUser()); } @Test @@ -256,6 +258,87 @@ public class TestApplicationLimits { assertEquals(0, queue.getNumPendingApplications(user_1)); } + @Test + public void testActiveLimitsWithKilledApps() throws Exception { + final String user_0 = "user_0"; + + int APPLICATION_ID = 0; + + // set max active to 2 + doReturn(2).when(queue).getMaximumActiveApplications(); + + // Submit first application + SchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0); + queue.submitApplication(app_0, user_0, A); + assertEquals(1, queue.getNumActiveApplications()); + assertEquals(0, queue.getNumPendingApplications()); + assertEquals(1, queue.getNumActiveApplications(user_0)); + assertEquals(0, queue.getNumPendingApplications(user_0)); + assertTrue(queue.activeApplications.contains(app_0)); + + // Submit second application + SchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0); + queue.submitApplication(app_1, user_0, A); + assertEquals(2, queue.getNumActiveApplications()); + assertEquals(0, queue.getNumPendingApplications()); + assertEquals(2, queue.getNumActiveApplications(user_0)); + assertEquals(0, queue.getNumPendingApplications(user_0)); + assertTrue(queue.activeApplications.contains(app_1)); + + // Submit third application, should remain pending + SchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0); + queue.submitApplication(app_2, user_0, A); + assertEquals(2, queue.getNumActiveApplications()); + assertEquals(1, queue.getNumPendingApplications()); + assertEquals(2, queue.getNumActiveApplications(user_0)); + assertEquals(1, queue.getNumPendingApplications(user_0)); + assertTrue(queue.pendingApplications.contains(app_2)); + + // Submit fourth application, should remain pending + SchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0); + queue.submitApplication(app_3, user_0, A); + assertEquals(2, queue.getNumActiveApplications()); + assertEquals(2, queue.getNumPendingApplications()); + assertEquals(2, queue.getNumActiveApplications(user_0)); + assertEquals(2, queue.getNumPendingApplications(user_0)); + assertTrue(queue.pendingApplications.contains(app_3)); + + // Kill 3rd pending application + queue.finishApplication(app_2, A); + assertEquals(2, queue.getNumActiveApplications()); + assertEquals(1, queue.getNumPendingApplications()); + assertEquals(2, queue.getNumActiveApplications(user_0)); + assertEquals(1, queue.getNumPendingApplications(user_0)); + assertFalse(queue.pendingApplications.contains(app_2)); + assertFalse(queue.activeApplications.contains(app_2)); + + // Finish 1st application, app_3 should become active + queue.finishApplication(app_0, A); + assertEquals(2, queue.getNumActiveApplications()); + assertEquals(0, queue.getNumPendingApplications()); + assertEquals(2, queue.getNumActiveApplications(user_0)); + assertEquals(0, queue.getNumPendingApplications(user_0)); + assertTrue(queue.activeApplications.contains(app_3)); + assertFalse(queue.pendingApplications.contains(app_3)); + assertFalse(queue.activeApplications.contains(app_0)); + + // Finish 2nd application + queue.finishApplication(app_1, A); + assertEquals(1, queue.getNumActiveApplications()); + assertEquals(0, queue.getNumPendingApplications()); + assertEquals(1, queue.getNumActiveApplications(user_0)); + assertEquals(0, queue.getNumPendingApplications(user_0)); + assertFalse(queue.activeApplications.contains(app_1)); + + // Finish 4th application + queue.finishApplication(app_3, A); + assertEquals(0, queue.getNumActiveApplications()); + assertEquals(0, queue.getNumPendingApplications()); + assertEquals(0, queue.getNumActiveApplications(user_0)); + assertEquals(0, queue.getNumPendingApplications(user_0)); + assertFalse(queue.activeApplications.contains(app_3)); + } + @Test public void testHeadroom() throws Exception { CapacitySchedulerConfiguration csConf = diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java index 9db0288ad58..52430e8bc1f 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java @@ -38,6 +38,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; @@ -57,6 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.junit.After; @@ -67,6 +70,8 @@ import org.mockito.stubbing.Answer; public class TestLeafQueue { + private static final Log LOG = LogFactory.getLog(TestLeafQueue.class); + private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); @@ -88,6 +93,7 @@ public class TestLeafQueue { csConf = new CapacitySchedulerConfiguration(); + csConf.setBoolean("yarn.scheduler.capacity.user-metrics.enable", true); setupQueueConfiguration(csConf); @@ -254,6 +260,35 @@ public class TestLeafQueue { assertEquals(7*GB, a.getMetrics().getAvailableMB()); } + @Test + public void testAppAttemptMetrics() throws Exception { + + // Manipulate queue 'a' + LeafQueue a = stubLeafQueue((LeafQueue) queues.get(B)); + + // Users + final String user_0 = "user_0"; + + // Submit applications + final ApplicationAttemptId appAttemptId_0 = TestUtils + .getMockApplicationAttemptId(0, 1); + SchedulerApp app_0 = new SchedulerApp(appAttemptId_0, user_0, a, null, + rmContext, null); + a.submitApplication(app_0, user_0, B); + + // Attempt the same application again + final ApplicationAttemptId appAttemptId_1 = TestUtils + .getMockApplicationAttemptId(0, 2); + SchedulerApp app_1 = new SchedulerApp(appAttemptId_1, user_0, a, null, + rmContext, null); + a.submitApplication(app_1, user_0, B); // same user + + assertEquals(1, a.getMetrics().getAppsSubmitted()); + assertEquals(1, a.getMetrics().getAppsPending()); + + QueueMetrics userMetrics = a.getMetrics().getUserMetrics(user_0); + assertEquals(1, userMetrics.getAppsSubmitted()); + } @Test public void testSingleQueueWithOneUser() throws Exception { @@ -472,6 +507,115 @@ public class TestLeafQueue { assertEquals(2*GB, app_1.getCurrentConsumption().getMemory()); } + @Test + public void testHeadroomWithMaxCap() throws Exception { + // Mock the queue + LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A)); + //unset maxCapacity + a.setMaxCapacity(1.0f); + + // Users + final String user_0 = "user_0"; + final String user_1 = "user_1"; + + // Submit applications + final ApplicationAttemptId appAttemptId_0 = + TestUtils.getMockApplicationAttemptId(0, 0); + SchedulerApp app_0 = + new SchedulerApp(appAttemptId_0, user_0, a, + a.getActiveUsersManager(), rmContext, null); + a.submitApplication(app_0, user_0, A); + + final ApplicationAttemptId appAttemptId_1 = + TestUtils.getMockApplicationAttemptId(1, 0); + SchedulerApp app_1 = + new SchedulerApp(appAttemptId_1, user_0, a, + a.getActiveUsersManager(), rmContext, null); + a.submitApplication(app_1, user_0, A); // same user + + final ApplicationAttemptId appAttemptId_2 = + TestUtils.getMockApplicationAttemptId(2, 0); + SchedulerApp app_2 = + new SchedulerApp(appAttemptId_2, user_1, a, + a.getActiveUsersManager(), rmContext, null); + a.submitApplication(app_2, user_1, A); + + // Setup some nodes + String host_0 = "host_0"; + SchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); + String host_1 = "host_1"; + SchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB); + + final int numNodes = 2; + Resource clusterResource = Resources.createResource(numNodes * (8*GB)); + when(csContext.getNumClusterNodes()).thenReturn(numNodes); + + // Setup resource-requests + Priority priority = TestUtils.createMockPriority(1); + app_0.updateResourceRequests(Collections.singletonList( + TestUtils.createResourceRequest(RMNodeImpl.ANY, 2*GB, 1, priority, + recordFactory))); + + app_1.updateResourceRequests(Collections.singletonList( + TestUtils.createResourceRequest(RMNodeImpl.ANY, 1*GB, 2, priority, + recordFactory))); + + /** + * Start testing... + */ + + // Set user-limit + a.setUserLimit(50); + a.setUserLimitFactor(2); + + // Now, only user_0 should be active since he is the only one with + // outstanding requests + assertEquals("There should only be 1 active user!", + 1, a.getActiveUsersManager().getNumActiveUsers()); + + // 1 container to user_0 + a.assignContainers(clusterResource, node_0); + assertEquals(2*GB, a.getUsedResources().getMemory()); + assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); + assertEquals(0*GB, app_1.getCurrentConsumption().getMemory()); + assertEquals(0*GB, app_0.getHeadroom().getMemory()); // User limit = 2G + assertEquals(0*GB, app_0.getHeadroom().getMemory()); // User limit = 2G + + // Again one to user_0 since he hasn't exceeded user limit yet + a.assignContainers(clusterResource, node_0); + assertEquals(3*GB, a.getUsedResources().getMemory()); + assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); + assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); + assertEquals(0*GB, app_0.getHeadroom().getMemory()); // 3G - 2G + assertEquals(0*GB, app_0.getHeadroom().getMemory()); // 3G - 2G + + // Submit requests for app_1 and set max-cap + a.setMaxCapacity(.1f); + app_2.updateResourceRequests(Collections.singletonList( + TestUtils.createResourceRequest(RMNodeImpl.ANY, 1*GB, 1, priority, + recordFactory))); + assertEquals(2, a.getActiveUsersManager().getNumActiveUsers()); + + // No more to user_0 since he is already over user-limit + // and no more containers to queue since it's already at max-cap + a.assignContainers(clusterResource, node_1); + assertEquals(3*GB, a.getUsedResources().getMemory()); + assertEquals(2*GB, app_0.getCurrentConsumption().getMemory()); + assertEquals(1*GB, app_1.getCurrentConsumption().getMemory()); + assertEquals(0*GB, app_2.getCurrentConsumption().getMemory()); + assertEquals(0*GB, app_0.getHeadroom().getMemory()); + assertEquals(0*GB, app_1.getHeadroom().getMemory()); + + // Check headroom for app_2 + LOG.info("here"); + app_1.updateResourceRequests(Collections.singletonList( // unset + TestUtils.createResourceRequest(RMNodeImpl.ANY, 1*GB, 0, priority, + recordFactory))); + assertEquals(1, a.getActiveUsersManager().getNumActiveUsers()); + a.assignContainers(clusterResource, node_1); + assertEquals(1*GB, app_2.getHeadroom().getMemory()); // hit queue max-cap + } + @Test public void testSingleQueueWithMultipleUsers() throws Exception { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java index 6bddf87eeb2..f3092b0637e 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java @@ -86,7 +86,7 @@ public class TestParentQueue { private SchedulerApp getMockApplication(int appId, String user) { SchedulerApp application = mock(SchedulerApp.class); doReturn(user).when(application).getUser(); - doReturn(null).when(application).getHeadroom(); + doReturn(Resources.createResource(0)).when(application).getHeadroom(); return application; } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java index 81d2d3d0fcf..a50d4c21700 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java @@ -26,17 +26,28 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetworkTopology; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.server.resourcemanager.Application; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.Task; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store; import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory; import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; +import org.apache.hadoop.yarn.server.resourcemanager.resourcetracker.InlineDispatcher; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.util.BuilderUtils; import org.junit.After; import org.junit.Before; +import org.junit.Test; public class TestFifoScheduler { private static final Log LOG = LogFactory.getLog(TestFifoScheduler.class); @@ -63,7 +74,30 @@ public class TestFifoScheduler { .getRMContext()); } + @Test + public void testAppAttemptMetrics() throws Exception { + AsyncDispatcher dispatcher = new InlineDispatcher(); + RMContext rmContext = new RMContextImpl(null, dispatcher, null, null, null); + FifoScheduler schedular = new FifoScheduler(); + schedular.reinitialize(new Configuration(), null, rmContext); + + ApplicationId appId = BuilderUtils.newApplicationId(200, 1); + ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId( + appId, 1); + + SchedulerEvent event = new AppAddedSchedulerEvent(appAttemptId, "queue", + "user"); + schedular.handle(event); + + appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2); + + event = new AppAddedSchedulerEvent(appAttemptId, "queue", "user"); + schedular.handle(event); + + QueueMetrics metrics = schedular.getRootQueueMetrics(); + Assert.assertEquals(1, metrics.getAppsSubmitted()); + } // @Test public void testFifoScheduler() throws Exception { diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java index a32c285eec8..5d97adae5d7 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java @@ -22,6 +22,7 @@ import java.io.PrintWriter; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState; import org.apache.hadoop.yarn.server.resourcemanager.webapp.NodesPage.NodesBlock; import org.apache.hadoop.yarn.webapp.test.WebAppTests; import org.junit.Before; @@ -39,7 +40,12 @@ import com.google.inject.Module; public class TestNodesPage { final int numberOfRacks = 2; - final int numberOfNodesPerRack = 2; + final int numberOfNodesPerRack = 6; + // The following is because of the way TestRMWebApp.mockRMContext creates + // nodes. + final int numberOfLostNodesPerRack = numberOfNodesPerRack + / RMNodeState.values().length; + // Number of Actual Table Headers for NodesPage.NodesBlock might change in // future. In that case this value should be adjusted to the new value. final int numberOfThInMetricsTable = 10; @@ -49,20 +55,22 @@ public class TestNodesPage { @Before public void setUp() throws Exception { - injector = WebAppTests.createMockInjector(RMContext.class, TestRMWebApp - .mockRMContext(3, numberOfRacks, numberOfNodesPerRack, - 8 * TestRMWebApp.GiB), new Module() { - @Override - public void configure(Binder binder) { - try { - binder.bind(ResourceManager.class).toInstance( - TestRMWebApp.mockRm(3, numberOfRacks, numberOfNodesPerRack, - 8 * TestRMWebApp.GiB)); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - }); + final RMContext mockRMContext = + TestRMWebApp.mockRMContext(3, numberOfRacks, numberOfNodesPerRack, + 8 * TestRMWebApp.GiB); + injector = + WebAppTests.createMockInjector(RMContext.class, mockRMContext, + new Module() { + @Override + public void configure(Binder binder) { + try { + binder.bind(ResourceManager.class).toInstance( + TestRMWebApp.mockRm(mockRMContext)); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + }); } @Test @@ -94,7 +102,7 @@ public class TestNodesPage { .print(" lostNodes = MockNodes.lostNodes(racks, numNodes, - newResource(mbsPerNode)); - final ConcurrentMap lostNodesMap = Maps.newConcurrentMap(); - for (RMNode node : lostNodes) { - lostNodesMap.put(node.getHostName(), node); + final List deactivatedNodes = + MockNodes.deactivatedNodes(racks, numNodes, newResource(mbsPerNode)); + final ConcurrentMap deactivatedNodesMap = + Maps.newConcurrentMap(); + for (RMNode node : deactivatedNodes) { + deactivatedNodesMap.put(node.getHostName(), node); } return new RMContextImpl(new MemStore(), null, null, null, null) { @Override @@ -138,7 +158,7 @@ public class TestRMWebApp { } @Override public ConcurrentMap getInactiveRMNodes() { - return lostNodesMap; + return deactivatedNodesMap; } @Override public ConcurrentMap getRMNodes() { @@ -149,9 +169,13 @@ public class TestRMWebApp { public static ResourceManager mockRm(int apps, int racks, int nodes, int mbsPerNode) throws IOException { - ResourceManager rm = mock(ResourceManager.class); RMContext rmContext = mockRMContext(apps, racks, nodes, - mbsPerNode); + mbsPerNode); + return mockRm(rmContext); + } + + public static ResourceManager mockRm(RMContext rmContext) throws IOException { + ResourceManager rm = mock(ResourceManager.class); ResourceScheduler rs = mockCapacityScheduler(); when(rm.getResourceScheduler()).thenReturn(rs); when(rm.getRMContext()).thenReturn(rmContext); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java index 8886d6e451a..04b4ad09241 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java @@ -55,6 +55,8 @@ import org.w3c.dom.Element; import org.w3c.dom.NodeList; import org.xml.sax.InputSource; +import clover.org.jfree.util.Log; + import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.servlet.GuiceServletContextListener; @@ -123,6 +125,46 @@ public class TestRMWebServicesNodes extends JerseyTest { testNodesHelper("nodes/", ""); } + @Test + public void testNodesDefaultWithUnHealthyNode() throws JSONException, + Exception { + + WebResource r = resource(); + MockNM nm1 = rm.registerNode("h1:1234", 5120); + MockNM nm2 = rm.registerNode("h2:1235", 5121); + rm.sendNodeStarted(nm1); + rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING); + rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW); + + // One unhealthy node which should not appear in the list after + // MAPREDUCE-3760. + MockNM nm3 = rm.registerNode("h3:1236", 5122); + rm.NMwaitForState(nm3.getNodeId(), RMNodeState.NEW); + rm.sendNodeStarted(nm3); + rm.NMwaitForState(nm3.getNodeId(), RMNodeState.RUNNING); + RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes() + .get(nm3.getNodeId()); + NodeHealthStatus nodeHealth = node.getNodeHealthStatus(); + nodeHealth.setHealthReport("test health report"); + nodeHealth.setIsNodeHealthy(false); + node.handle(new RMNodeStatusEvent(nm3.getNodeId(), nodeHealth, + new ArrayList(), null, null)); + rm.NMwaitForState(nm3.getNodeId(), RMNodeState.UNHEALTHY); + + ClientResponse response = + r.path("ws").path("v1").path("cluster").path("nodes") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + + assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject nodes = json.getJSONObject("nodes"); + assertEquals("incorrect number of elements", 1, nodes.length()); + JSONArray nodeArray = nodes.getJSONArray("node"); + // Just 2 nodes, leaving behind the unhealthy node. + assertEquals("incorrect number of elements", 2, nodeArray.length()); + } + @Test public void testNodesQueryState() throws JSONException, Exception { WebResource r = resource(); diff --git a/hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/hdfs/server/datanode/RaidBlockSender.java b/hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/hdfs/server/datanode/RaidBlockSender.java index 8cde11d9cdf..c1fc998471d 100644 --- a/hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/hdfs/server/datanode/RaidBlockSender.java +++ b/hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/hdfs/server/datanode/RaidBlockSender.java @@ -108,7 +108,7 @@ public class RaidBlockSender implements java.io.Closeable { BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn); short version = header.getVersion(); - if (version != FSDataset.METADATA_VERSION) { + if (version != BlockMetadataHeader.VERSION) { LOG.warn("Wrong version (" + version + ") for metadata file for " + block + " ignoring ..."); } diff --git a/hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/raid/BlockFixer.java b/hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/raid/BlockFixer.java index 5759fb0cd7b..dabb73564a3 100644 --- a/hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/raid/BlockFixer.java +++ b/hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/raid/BlockFixer.java @@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; -import org.apache.hadoop.hdfs.server.datanode.FSDataset; +import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; import org.apache.hadoop.hdfs.server.datanode.RaidBlockSender; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -646,7 +646,7 @@ public abstract class BlockFixer extends Configured implements Runnable { DataOutputStream mdOut = new DataOutputStream(mdOutBase); // First, write out the version. - mdOut.writeShort(FSDataset.METADATA_VERSION); + mdOut.writeShort(BlockMetadataHeader.VERSION); // Create a summer and write out its header. int bytesPerChecksum = conf.getInt("io.bytes.per.checksum", 512); diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 55763f8fdbd..70cd9a45a34 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -662,7 +662,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 2.7 + 2.8.1 org.apache.maven.plugins diff --git a/pom.xml b/pom.xml index 3ecaf8832ba..1de33ae5391 100644 --- a/pom.xml +++ b/pom.xml @@ -187,7 +187,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 2.8 + 2.8.1 aggregate