Merge trunk into the HDFS-347 branch.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-347@1467511 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
b7aaa1b74c
|
@ -143,10 +143,6 @@
|
|||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>jetty</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>jetty-util</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.sun.jersey</groupId>
|
||||
<artifactId>jersey-core</artifactId>
|
||||
|
|
|
@ -165,6 +165,9 @@ Trunk (Unreleased)
|
|||
|
||||
BUG FIXES
|
||||
|
||||
HADOOP-9451. Fault single-layer config if node group topology is enabled.
|
||||
(Junping Du via llu)
|
||||
|
||||
HADOOP-8419. Fixed GzipCode NPE reset for IBM JDK. (Yu Li via eyang)
|
||||
|
||||
HADOOP-9041. FsUrlStreamHandlerFactory could cause an infinite loop in
|
||||
|
@ -503,6 +506,10 @@ Trunk (Unreleased)
|
|||
HADOOP-9353. Activate native-win maven profile by default on Windows.
|
||||
(Arpit Agarwal via szetszwo)
|
||||
|
||||
HADOOP-9437. TestNativeIO#testRenameTo fails on Windows due to assumption
|
||||
that POSIX errno is embedded in NativeIOException. (Chris Nauroth via
|
||||
suresh)
|
||||
|
||||
Release 2.0.5-beta - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -536,6 +543,9 @@ Release 2.0.5-beta - UNRELEASED
|
|||
|
||||
HADOOP-9358. "Auth failed" log should include exception string (todd)
|
||||
|
||||
HADOOP-9401. CodecPool: Add counters for number of (de)compressors
|
||||
leased out. (kkambatl via tucu)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs
|
||||
|
@ -604,8 +614,17 @@ Release 2.0.5-beta - UNRELEASED
|
|||
HADOOP-9125. LdapGroupsMapping threw CommunicationException after some
|
||||
idle time. (Kai Zheng via atm)
|
||||
|
||||
HADOOP-9357. Fallback to default authority if not specified in FileContext.
|
||||
(Andrew Wang via eli)
|
||||
HADOOP-9429. TestConfiguration fails with IBM JAVA. (Amir Sanjar via
|
||||
suresh)
|
||||
|
||||
HADOOP-9222. Cover package with org.apache.hadoop.io.lz4 unit tests (Vadim
|
||||
Bondarev via jlowe)
|
||||
|
||||
HADOOP-9233. Cover package org.apache.hadoop.io.compress.zlib with unit
|
||||
tests (Vadim Bondarev via jlowe)
|
||||
|
||||
HADOOP-9211. Set default max heap size in HADOOP_CLIENT_OPTS to 512m
|
||||
in order to avoid OOME. (Plamen Jeliazkov via shv)
|
||||
|
||||
Release 2.0.4-alpha - UNRELEASED
|
||||
|
||||
|
@ -619,6 +638,9 @@ Release 2.0.4-alpha - UNRELEASED
|
|||
|
||||
BUG FIXES
|
||||
|
||||
HADOOP-9467. Metrics2 record filter should check name as well as tags.
|
||||
(Ganeshan Iyler via llu)
|
||||
|
||||
HADOOP-9406. hadoop-client leaks dependency on JDK tools jar. (tucu)
|
||||
|
||||
HADOOP-9301. hadoop client servlet/jsp/jetty/tomcat JARs creating
|
||||
|
@ -630,6 +652,9 @@ Release 2.0.4-alpha - UNRELEASED
|
|||
HADOOP-9444. Modify hadoop-policy.xml to replace unexpanded variables to a
|
||||
default value of '*'. (Roman Shaposhnik via vinodkv)
|
||||
|
||||
HADOOP-9471. hadoop-client wrongfully excludes jetty-util JAR,
|
||||
breaking webhdfs. (tucu)
|
||||
|
||||
Release 2.0.3-alpha - 2013-02-06
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -1593,6 +1618,24 @@ Release 2.0.0-alpha - 05-23-2012
|
|||
HADOOP-8655. Fix TextInputFormat for large deliminators. (Gelesh via
|
||||
bobby)
|
||||
|
||||
Release 0.23.8 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
||||
HADOOP-9222. Cover package with org.apache.hadoop.io.lz4 unit tests (Vadim
|
||||
Bondarev via jlowe)
|
||||
|
||||
HADOOP-9233. Cover package org.apache.hadoop.io.compress.zlib with unit
|
||||
tests (Vadim Bondarev via jlowe)
|
||||
|
||||
Release 0.23.7 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -62,7 +62,7 @@ export HADOOP_DATANODE_OPTS="-Dhadoop.security.logger=ERROR,RFAS $HADOOP_DATANOD
|
|||
export HADOOP_SECONDARYNAMENODE_OPTS="-Dhadoop.security.logger=${HADOOP_SECURITY_LOGGER:-INFO,RFAS} -Dhdfs.audit.logger=${HDFS_AUDIT_LOGGER:-INFO,NullAppender} $HADOOP_SECONDARYNAMENODE_OPTS"
|
||||
|
||||
# The following applies to multiple commands (fs, dfs, fsck, distcp etc)
|
||||
export HADOOP_CLIENT_OPTS="-Xmx128m $HADOOP_CLIENT_OPTS"
|
||||
export HADOOP_CLIENT_OPTS="-Xmx512m $HADOOP_CLIENT_OPTS"
|
||||
#HADOOP_JAVA_PLATFORM_OPTS="-XX:-UsePerfData $HADOOP_JAVA_PLATFORM_OPTS"
|
||||
|
||||
# On secure datanodes, user to run the datanode as after dropping privileges
|
||||
|
|
|
@ -244,33 +244,17 @@ public final class FileContext {
|
|||
}
|
||||
|
||||
/*
|
||||
* Resolve a relative path passed from the user.
|
||||
*
|
||||
* Relative paths are resolved against the current working directory
|
||||
* (e.g. "foo/bar" becomes "/<workingDir>/foo/bar").
|
||||
* Fully-qualified URIs (e.g. "hdfs://nn:p/foo/bar") and slash-relative paths
|
||||
* Remove relative part - return "absolute":
|
||||
* If input is relative path ("foo/bar") add wd: ie "/<workingDir>/foo/bar"
|
||||
* A fully qualified uri ("hdfs://nn:p/foo/bar") or a slash-relative path
|
||||
* ("/foo/bar") are returned unchanged.
|
||||
*
|
||||
* Additionally, we fix malformed URIs that specify a scheme but not an
|
||||
* authority (e.g. "hdfs:///foo/bar"). Per RFC 2395, we remove the scheme
|
||||
* if it matches the default FS, and let the default FS add in the default
|
||||
* scheme and authority later (see {@link #AbstractFileSystem#checkPath}).
|
||||
*
|
||||
* Applications that use FileContext should use #makeQualified() since
|
||||
* they really want a fully-qualified URI.
|
||||
* they really want a fully qualified URI.
|
||||
* Hence this method is not called makeAbsolute() and
|
||||
* has been deliberately declared private.
|
||||
*/
|
||||
private Path fixRelativePart(Path p) {
|
||||
// Per RFC 2396 5.2, drop schema if there is a scheme but no authority.
|
||||
if (p.hasSchemeAndNoAuthority()) {
|
||||
String scheme = p.toUri().getScheme();
|
||||
if (scheme.equalsIgnoreCase(defaultFS.getUri().getScheme())) {
|
||||
p = new Path(p.toUri().getSchemeSpecificPart());
|
||||
}
|
||||
}
|
||||
// Absolute paths are unchanged. Relative paths are resolved against the
|
||||
// current working directory.
|
||||
if (p.isUriPathAbsolute()) {
|
||||
return p;
|
||||
} else {
|
||||
|
|
|
@ -257,10 +257,6 @@ public class Path implements Comparable {
|
|||
uri.getScheme() == null && uri.getAuthority() == null);
|
||||
}
|
||||
|
||||
public boolean hasSchemeAndNoAuthority() {
|
||||
return uri.getScheme() != null && uri.getAuthority() == null;
|
||||
}
|
||||
|
||||
/**
|
||||
* True if the path component (i.e. directory) of this URI is absolute.
|
||||
*/
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.util.ArrayList;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -29,6 +30,10 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.cache.CacheBuilder;
|
||||
import com.google.common.cache.CacheLoader;
|
||||
import com.google.common.cache.LoadingCache;
|
||||
|
||||
/**
|
||||
* A global compressor/decompressor pool used to save and reuse
|
||||
* (possibly native) compression/decompression codecs.
|
||||
|
@ -52,6 +57,29 @@ public class CodecPool {
|
|||
private static final Map<Class<Decompressor>, List<Decompressor>> decompressorPool =
|
||||
new HashMap<Class<Decompressor>, List<Decompressor>>();
|
||||
|
||||
private static <T> LoadingCache<Class<T>, AtomicInteger> createCache(
|
||||
Class<T> klass) {
|
||||
return CacheBuilder.newBuilder().build(
|
||||
new CacheLoader<Class<T>, AtomicInteger>() {
|
||||
@Override
|
||||
public AtomicInteger load(Class<T> key) throws Exception {
|
||||
return new AtomicInteger();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Map to track the number of leased compressors
|
||||
*/
|
||||
private static final LoadingCache<Class<Compressor>, AtomicInteger> compressorCounts =
|
||||
createCache(Compressor.class);
|
||||
|
||||
/**
|
||||
* Map to tracks the number of leased decompressors
|
||||
*/
|
||||
private static final LoadingCache<Class<Decompressor>, AtomicInteger> decompressorCounts =
|
||||
createCache(Decompressor.class);
|
||||
|
||||
private static <T> T borrow(Map<Class<T>, List<T>> pool,
|
||||
Class<? extends T> codecClass) {
|
||||
T codec = null;
|
||||
|
@ -90,6 +118,21 @@ public class CodecPool {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private static <T> int getLeaseCount(
|
||||
LoadingCache<Class<T>, AtomicInteger> usageCounts,
|
||||
Class<? extends T> codecClass) {
|
||||
return usageCounts.getUnchecked((Class<T>) codecClass).get();
|
||||
}
|
||||
|
||||
private static <T> void updateLeaseCount(
|
||||
LoadingCache<Class<T>, AtomicInteger> usageCounts, T codec, int delta) {
|
||||
if (codec != null) {
|
||||
Class<T> codecClass = ReflectionUtils.getClass(codec);
|
||||
usageCounts.getUnchecked(codecClass).addAndGet(delta);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a {@link Compressor} for the given {@link CompressionCodec} from the
|
||||
* pool or a new one.
|
||||
|
@ -111,6 +154,7 @@ public class CodecPool {
|
|||
LOG.debug("Got recycled compressor");
|
||||
}
|
||||
}
|
||||
updateLeaseCount(compressorCounts, compressor, 1);
|
||||
return compressor;
|
||||
}
|
||||
|
||||
|
@ -137,6 +181,7 @@ public class CodecPool {
|
|||
LOG.debug("Got recycled decompressor");
|
||||
}
|
||||
}
|
||||
updateLeaseCount(decompressorCounts, decompressor, 1);
|
||||
return decompressor;
|
||||
}
|
||||
|
||||
|
@ -155,6 +200,7 @@ public class CodecPool {
|
|||
}
|
||||
compressor.reset();
|
||||
payback(compressorPool, compressor);
|
||||
updateLeaseCount(compressorCounts, compressor, -1);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -173,5 +219,24 @@ public class CodecPool {
|
|||
}
|
||||
decompressor.reset();
|
||||
payback(decompressorPool, decompressor);
|
||||
updateLeaseCount(decompressorCounts, decompressor, -1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the number of leased {@link Compressor}s for this
|
||||
* {@link CompressionCodec}
|
||||
*/
|
||||
public static int getLeasedCompressorsCount(CompressionCodec codec) {
|
||||
return (codec == null) ? 0 : getLeaseCount(compressorCounts,
|
||||
codec.getCompressorType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the number of leased {@link Decompressor}s for this
|
||||
* {@link CompressionCodec}
|
||||
*/
|
||||
public static int getLeasedDecompressorsCount(CompressionCodec codec) {
|
||||
return (codec == null) ? 0 : getLeaseCount(decompressorCounts,
|
||||
codec.getDecompressorType());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,7 +57,7 @@ public abstract class MetricsFilter implements MetricsPlugin {
|
|||
* @return true to accept; false otherwise.
|
||||
*/
|
||||
public boolean accepts(MetricsRecord record) {
|
||||
return accepts(record.tags());
|
||||
return accepts(record.name()) && accepts(record.tags());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -191,7 +191,12 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
|
|||
}
|
||||
rack = getNode(nodeGroup.getNetworkLocation());
|
||||
|
||||
if (rack != null && !(rack instanceof InnerNode)) {
|
||||
// rack should be an innerNode and with parent.
|
||||
// note: rack's null parent case is: node's topology only has one layer,
|
||||
// so rack is recognized as "/" and no parent.
|
||||
// This will be recognized as a node with fault topology.
|
||||
if (rack != null &&
|
||||
(!(rack instanceof InnerNode) || rack.getParent() == null)) {
|
||||
throw new IllegalArgumentException("Unexpected data node "
|
||||
+ node.toString()
|
||||
+ " at an illegal network location");
|
||||
|
|
|
@ -27,8 +27,10 @@ import java.io.DataInputStream;
|
|||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import javax.crypto.SecretKey;
|
||||
|
||||
|
@ -144,6 +146,10 @@ extends AbstractDelegationTokenIdentifier>
|
|||
return;
|
||||
}
|
||||
|
||||
protected void logExpireToken(TokenIdent ident) throws IOException {
|
||||
return;
|
||||
}
|
||||
|
||||
/**
|
||||
* Update the current master key
|
||||
* This is called once by startThreads before tokenRemoverThread is created,
|
||||
|
@ -363,16 +369,26 @@ extends AbstractDelegationTokenIdentifier>
|
|||
}
|
||||
|
||||
/** Remove expired delegation tokens from cache */
|
||||
private synchronized void removeExpiredToken() {
|
||||
private void removeExpiredToken() throws IOException {
|
||||
long now = Time.now();
|
||||
Iterator<DelegationTokenInformation> i = currentTokens.values().iterator();
|
||||
Set<TokenIdent> expiredTokens = new HashSet<TokenIdent>();
|
||||
synchronized (this) {
|
||||
Iterator<Map.Entry<TokenIdent, DelegationTokenInformation>> i =
|
||||
currentTokens.entrySet().iterator();
|
||||
while (i.hasNext()) {
|
||||
long renewDate = i.next().getRenewDate();
|
||||
if (now > renewDate) {
|
||||
Map.Entry<TokenIdent, DelegationTokenInformation> entry = i.next();
|
||||
long renewDate = entry.getValue().getRenewDate();
|
||||
if (renewDate < now) {
|
||||
expiredTokens.add(entry.getKey());
|
||||
i.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
// don't hold lock on 'this' to avoid edit log updates blocking token ops
|
||||
for (TokenIdent ident : expiredTokens) {
|
||||
logExpireToken(ident);
|
||||
}
|
||||
}
|
||||
|
||||
public void stopThreads() {
|
||||
if (LOG.isDebugEnabled())
|
||||
|
|
|
@ -816,6 +816,7 @@ JNIEXPORT void JNICALL
|
|||
Java_org_apache_hadoop_io_nativeio_NativeIO_renameTo0(JNIEnv *env,
|
||||
jclass clazz, jstring jsrc, jstring jdst)
|
||||
{
|
||||
#ifdef UNIX
|
||||
const char *src = NULL, *dst = NULL;
|
||||
|
||||
src = (*env)->GetStringUTFChars(env, jsrc, NULL);
|
||||
|
@ -829,6 +830,23 @@ jclass clazz, jstring jsrc, jstring jdst)
|
|||
done:
|
||||
if (src) (*env)->ReleaseStringUTFChars(env, jsrc, src);
|
||||
if (dst) (*env)->ReleaseStringUTFChars(env, jdst, dst);
|
||||
#endif
|
||||
|
||||
#ifdef WINDOWS
|
||||
LPCWSTR src = NULL, dst = NULL;
|
||||
|
||||
src = (LPCWSTR) (*env)->GetStringChars(env, jsrc, NULL);
|
||||
if (!src) goto done; // exception was thrown
|
||||
dst = (LPCWSTR) (*env)->GetStringChars(env, jdst, NULL);
|
||||
if (!dst) goto done; // exception was thrown
|
||||
if (!MoveFile(src, dst)) {
|
||||
throw_ioe(env, GetLastError());
|
||||
}
|
||||
|
||||
done:
|
||||
if (src) (*env)->ReleaseStringChars(env, jsrc, src);
|
||||
if (dst) (*env)->ReleaseStringChars(env, jdst, dst);
|
||||
#endif
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -249,7 +249,7 @@ Hadoop MapReduce Next Generation - Cluster Setup
|
|||
*-------------------------+-------------------------+------------------------+
|
||||
| <<<yarn.resourcemanager.scheduler.class>>> | | |
|
||||
| | <<<ResourceManager>>> Scheduler class. | |
|
||||
| | | <<<CapacityScheduler>>> (recommended) or <<<FifoScheduler>>> |
|
||||
| | | <<<CapacityScheduler>>> (recommended), <<<FairScheduler>>> (also recommended), or <<<FifoScheduler>>> |
|
||||
*-------------------------+-------------------------+------------------------+
|
||||
| <<<yarn.scheduler.minimum-allocation-mb>>> | | |
|
||||
| | Minimum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |
|
||||
|
|
|
@ -51,6 +51,10 @@ public class TestConfiguration extends TestCase {
|
|||
final static String CONFIG = new File("./test-config.xml").getAbsolutePath();
|
||||
final static String CONFIG2 = new File("./test-config2.xml").getAbsolutePath();
|
||||
final static Random RAN = new Random();
|
||||
final static boolean IBMJAVA = System.getProperty("java.vendor").contains("IBM");
|
||||
final static String XMLHEADER =
|
||||
IBMJAVA?"<?xml version=\"1.0\" encoding=\"UTF-8\"?><configuration>":
|
||||
"<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?><configuration>";
|
||||
|
||||
@Override
|
||||
protected void setUp() throws Exception {
|
||||
|
@ -327,8 +331,8 @@ public class TestConfiguration extends TestCase {
|
|||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
conf.writeXml(baos);
|
||||
String result = baos.toString();
|
||||
assertTrue("Result has proper header", result.startsWith(
|
||||
"<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?><configuration>"));
|
||||
assertTrue("Result has proper header", result.startsWith(XMLHEADER));
|
||||
|
||||
assertTrue("Result has proper footer", result.endsWith("</configuration>"));
|
||||
}
|
||||
|
||||
|
|
|
@ -21,8 +21,6 @@ package org.apache.hadoop.fs;
|
|||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.EnumSet;
|
||||
|
||||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||
|
@ -1167,40 +1165,6 @@ public abstract class FileContextMainOperationsBaseTest {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that URIs with a scheme, no authority, and absolute path component
|
||||
* resolve with the authority of the default FS.
|
||||
*/
|
||||
@Test(timeout=30000)
|
||||
public void testAbsolutePathSchemeNoAuthority() throws IOException,
|
||||
URISyntaxException {
|
||||
Path file = getTestRootPath(fc, "test/file");
|
||||
createFile(file);
|
||||
URI uri = file.toUri();
|
||||
URI noAuthorityUri = new URI(uri.getScheme(), null, uri.getPath(),
|
||||
uri.getQuery(), uri.getFragment());
|
||||
Path noAuthority = new Path(noAuthorityUri);
|
||||
Assert.assertEquals(fc.getFileStatus(file), fc.getFileStatus(noAuthority));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that URIs with a scheme, no authority, and relative path component
|
||||
* resolve with the authority of the default FS.
|
||||
*/
|
||||
@Test(timeout=30000)
|
||||
public void testRelativePathSchemeNoAuthority() throws IOException,
|
||||
URISyntaxException {
|
||||
Path workDir = new Path(getAbsoluteTestRootPath(fc), new Path("test"));
|
||||
fc.setWorkingDirectory(workDir);
|
||||
Path file = new Path(workDir, "file");
|
||||
createFile(file);
|
||||
URI uri = file.toUri();
|
||||
URI noAuthorityUri = new URI(uri.getScheme() + ":file");
|
||||
System.out.println(noAuthorityUri);
|
||||
Path noAuthority = new Path(noAuthorityUri);
|
||||
Assert.assertEquals(fc.getFileStatus(file), fc.getFileStatus(noAuthority));
|
||||
}
|
||||
|
||||
protected void createFile(Path path) throws IOException {
|
||||
FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
|
||||
Options.CreateOpts.createParent());
|
||||
|
|
|
@ -0,0 +1,524 @@
|
|||
/*
|
||||
* 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.io.compress;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.compress.lz4.Lz4Compressor;
|
||||
import org.apache.hadoop.io.compress.snappy.SnappyCompressor;
|
||||
import org.apache.hadoop.io.compress.zlib.BuiltInZlibDeflater;
|
||||
import org.apache.hadoop.io.compress.zlib.ZlibCompressor;
|
||||
import org.apache.hadoop.io.compress.zlib.ZlibFactory;
|
||||
import org.apache.hadoop.util.NativeCodeLoader;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.Assert;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class CompressDecompressTester<T extends Compressor, E extends Decompressor> {
|
||||
|
||||
private static final Logger logger = Logger
|
||||
.getLogger(CompressDecompressTester.class);
|
||||
|
||||
private final byte[] originalRawData;
|
||||
|
||||
private ImmutableList<TesterPair<T, E>> pairs = ImmutableList.of();
|
||||
private ImmutableList.Builder<TesterPair<T, E>> builder = ImmutableList.builder();
|
||||
|
||||
private ImmutableSet<CompressionTestStrategy> stateges = ImmutableSet.of();
|
||||
|
||||
private PreAssertionTester<T, E> assertionDelegate;
|
||||
|
||||
public CompressDecompressTester(byte[] originalRawData) {
|
||||
this.originalRawData = Arrays.copyOf(originalRawData,
|
||||
originalRawData.length);
|
||||
this.assertionDelegate = new PreAssertionTester<T, E>() {
|
||||
|
||||
@Override
|
||||
public ImmutableList<TesterPair<T, E>> filterOnAssumeWhat(
|
||||
ImmutableList<TesterPair<T, E>> pairs) {
|
||||
ImmutableList.Builder<TesterPair<T, E>> builder = ImmutableList
|
||||
.builder();
|
||||
|
||||
for (TesterPair<T, E> pair : pairs) {
|
||||
if (isAvailable(pair))
|
||||
builder.add(pair);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private static boolean isNativeSnappyLoadable() {
|
||||
boolean snappyAvailable = false;
|
||||
boolean loaded = false;
|
||||
try {
|
||||
System.loadLibrary("snappy");
|
||||
logger.warn("Snappy native library is available");
|
||||
snappyAvailable = true;
|
||||
boolean hadoopNativeAvailable = NativeCodeLoader.isNativeCodeLoaded();
|
||||
loaded = snappyAvailable && hadoopNativeAvailable;
|
||||
if (loaded) {
|
||||
logger.info("Snappy native library loaded");
|
||||
} else {
|
||||
logger.warn("Snappy native library not loaded");
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
logger.warn("Failed to load snappy: ", t);
|
||||
return false;
|
||||
}
|
||||
return loaded;
|
||||
}
|
||||
|
||||
public static <T extends Compressor, E extends Decompressor> CompressDecompressTester<T, E> of(
|
||||
byte[] rawData) {
|
||||
return new CompressDecompressTester<T, E>(rawData);
|
||||
}
|
||||
|
||||
|
||||
public CompressDecompressTester<T, E> withCompressDecompressPair(
|
||||
T compressor, E decompressor) {
|
||||
addPair(
|
||||
compressor,
|
||||
decompressor,
|
||||
Joiner.on("_").join(compressor.getClass().getCanonicalName(),
|
||||
decompressor.getClass().getCanonicalName()));
|
||||
return this;
|
||||
}
|
||||
|
||||
public CompressDecompressTester<T, E> withTestCases(
|
||||
ImmutableSet<CompressionTestStrategy> stateges) {
|
||||
this.stateges = ImmutableSet.copyOf(stateges);
|
||||
return this;
|
||||
}
|
||||
|
||||
private void addPair(T compressor, E decompressor, String name) {
|
||||
builder.add(new TesterPair<T, E>(name, compressor, decompressor));
|
||||
}
|
||||
|
||||
public void test() throws InstantiationException, IllegalAccessException {
|
||||
pairs = builder.build();
|
||||
pairs = assertionDelegate.filterOnAssumeWhat(pairs);
|
||||
|
||||
for (TesterPair<T, E> pair : pairs) {
|
||||
for (CompressionTestStrategy strategy : stateges) {
|
||||
strategy.getTesterStrategy().assertCompression(pair.getName(),
|
||||
pair.getCompressor(), pair.getDecompressor(),
|
||||
Arrays.copyOf(originalRawData, originalRawData.length));
|
||||
}
|
||||
}
|
||||
endAll(pairs);
|
||||
}
|
||||
|
||||
private void endAll(ImmutableList<TesterPair<T, E>> pairs) {
|
||||
for (TesterPair<T, E> pair : pairs)
|
||||
pair.end();
|
||||
}
|
||||
|
||||
interface PreAssertionTester<T extends Compressor, E extends Decompressor> {
|
||||
ImmutableList<TesterPair<T, E>> filterOnAssumeWhat(
|
||||
ImmutableList<TesterPair<T, E>> pairs);
|
||||
}
|
||||
|
||||
public enum CompressionTestStrategy {
|
||||
|
||||
COMPRESS_DECOMPRESS_ERRORS(new TesterCompressionStrategy() {
|
||||
private final Joiner joiner = Joiner.on("- ");
|
||||
|
||||
@Override
|
||||
public void assertCompression(String name, Compressor compressor,
|
||||
Decompressor decompressor, byte[] rawData) {
|
||||
assertTrue(checkSetInputNullPointerException(compressor));
|
||||
assertTrue(checkSetInputNullPointerException(decompressor));
|
||||
|
||||
assertTrue(checkCompressArrayIndexOutOfBoundsException(compressor,
|
||||
rawData));
|
||||
assertTrue(checkCompressArrayIndexOutOfBoundsException(decompressor,
|
||||
rawData));
|
||||
|
||||
assertTrue(checkCompressNullPointerException(compressor, rawData));
|
||||
assertTrue(checkCompressNullPointerException(decompressor, rawData));
|
||||
|
||||
assertTrue(checkSetInputArrayIndexOutOfBoundsException(compressor));
|
||||
assertTrue(checkSetInputArrayIndexOutOfBoundsException(decompressor));
|
||||
}
|
||||
|
||||
private boolean checkSetInputNullPointerException(Compressor compressor) {
|
||||
try {
|
||||
compressor.setInput(null, 0, 1);
|
||||
} catch (NullPointerException npe) {
|
||||
return true;
|
||||
} catch (Exception ex) {
|
||||
logger.error(joiner.join(compressor.getClass().getCanonicalName(),
|
||||
"checkSetInputNullPointerException error !!!"));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkCompressNullPointerException(Compressor compressor,
|
||||
byte[] rawData) {
|
||||
try {
|
||||
compressor.setInput(rawData, 0, rawData.length);
|
||||
compressor.compress(null, 0, 1);
|
||||
} catch (NullPointerException npe) {
|
||||
return true;
|
||||
} catch (Exception ex) {
|
||||
logger.error(joiner.join(compressor.getClass().getCanonicalName(),
|
||||
"checkCompressNullPointerException error !!!"));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkCompressNullPointerException(
|
||||
Decompressor decompressor, byte[] rawData) {
|
||||
try {
|
||||
decompressor.setInput(rawData, 0, rawData.length);
|
||||
decompressor.decompress(null, 0, 1);
|
||||
} catch (NullPointerException npe) {
|
||||
return true;
|
||||
} catch (Exception ex) {
|
||||
logger.error(joiner.join(decompressor.getClass().getCanonicalName(),
|
||||
"checkCompressNullPointerException error !!!"));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkSetInputNullPointerException(
|
||||
Decompressor decompressor) {
|
||||
try {
|
||||
decompressor.setInput(null, 0, 1);
|
||||
} catch (NullPointerException npe) {
|
||||
return true;
|
||||
} catch (Exception ex) {
|
||||
logger.error(joiner.join(decompressor.getClass().getCanonicalName(),
|
||||
"checkSetInputNullPointerException error !!!"));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkSetInputArrayIndexOutOfBoundsException(
|
||||
Compressor compressor) {
|
||||
try {
|
||||
compressor.setInput(new byte[] { (byte) 0 }, 0, -1);
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error(joiner.join(compressor.getClass().getCanonicalName(),
|
||||
"checkSetInputArrayIndexOutOfBoundsException error !!!"));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkCompressArrayIndexOutOfBoundsException(
|
||||
Compressor compressor, byte[] rawData) {
|
||||
try {
|
||||
compressor.setInput(rawData, 0, rawData.length);
|
||||
compressor.compress(new byte[rawData.length], 0, -1);
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error(joiner.join(compressor.getClass().getCanonicalName(),
|
||||
"checkCompressArrayIndexOutOfBoundsException error !!!"));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkCompressArrayIndexOutOfBoundsException(
|
||||
Decompressor decompressor, byte[] rawData) {
|
||||
try {
|
||||
decompressor.setInput(rawData, 0, rawData.length);
|
||||
decompressor.decompress(new byte[rawData.length], 0, -1);
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error(joiner.join(decompressor.getClass().getCanonicalName(),
|
||||
"checkCompressArrayIndexOutOfBoundsException error !!!"));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkSetInputArrayIndexOutOfBoundsException(
|
||||
Decompressor decompressor) {
|
||||
try {
|
||||
decompressor.setInput(new byte[] { (byte) 0 }, 0, -1);
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error(joiner.join(decompressor.getClass().getCanonicalName(),
|
||||
"checkNullPointerException error !!!"));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
}),
|
||||
|
||||
COMPRESS_DECOMPRESS_SINGLE_BLOCK(new TesterCompressionStrategy() {
|
||||
final Joiner joiner = Joiner.on("- ");
|
||||
|
||||
@Override
|
||||
public void assertCompression(String name, Compressor compressor,
|
||||
Decompressor decompressor, byte[] rawData) {
|
||||
|
||||
int cSize = 0;
|
||||
int decompressedSize = 0;
|
||||
byte[] compressedResult = new byte[rawData.length];
|
||||
byte[] decompressedBytes = new byte[rawData.length];
|
||||
try {
|
||||
assertTrue(
|
||||
joiner.join(name, "compressor.needsInput before error !!!"),
|
||||
compressor.needsInput());
|
||||
assertTrue(
|
||||
joiner.join(name, "compressor.getBytesWritten before error !!!"),
|
||||
compressor.getBytesWritten() == 0);
|
||||
compressor.setInput(rawData, 0, rawData.length);
|
||||
compressor.finish();
|
||||
while (!compressor.finished()) {
|
||||
cSize += compressor.compress(compressedResult, 0,
|
||||
compressedResult.length);
|
||||
}
|
||||
compressor.reset();
|
||||
|
||||
assertTrue(
|
||||
joiner.join(name, "decompressor.needsInput() before error !!!"),
|
||||
decompressor.needsInput());
|
||||
decompressor.setInput(compressedResult, 0, cSize);
|
||||
assertFalse(
|
||||
joiner.join(name, "decompressor.needsInput() after error !!!"),
|
||||
decompressor.needsInput());
|
||||
while (!decompressor.finished()) {
|
||||
decompressedSize = decompressor.decompress(decompressedBytes, 0,
|
||||
decompressedBytes.length);
|
||||
}
|
||||
decompressor.reset();
|
||||
assertTrue(joiner.join(name, " byte size not equals error !!!"),
|
||||
decompressedSize == rawData.length);
|
||||
assertArrayEquals(
|
||||
joiner.join(name, " byte arrays not equals error !!!"), rawData,
|
||||
decompressedBytes);
|
||||
} catch (Exception ex) {
|
||||
fail(joiner.join(name, ex.getMessage()));
|
||||
}
|
||||
}
|
||||
}),
|
||||
|
||||
COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM(new TesterCompressionStrategy() {
|
||||
final Joiner joiner = Joiner.on("- ");
|
||||
final ImmutableMap<Class<? extends Compressor>, Integer> emptySize = ImmutableMap
|
||||
.of(Lz4Compressor.class, 4, ZlibCompressor.class, 16,
|
||||
SnappyCompressor.class, 4, BuiltInZlibDeflater.class, 16);
|
||||
|
||||
@Override
|
||||
void assertCompression(String name, Compressor compressor,
|
||||
Decompressor decompressor, byte[] originalRawData) {
|
||||
byte[] buf = null;
|
||||
ByteArrayInputStream bytesIn = null;
|
||||
BlockDecompressorStream blockDecompressorStream = null;
|
||||
ByteArrayOutputStream bytesOut = new ByteArrayOutputStream();
|
||||
// close without write
|
||||
try {
|
||||
compressor.reset();
|
||||
// decompressor.end();
|
||||
BlockCompressorStream blockCompressorStream = new BlockCompressorStream(
|
||||
bytesOut, compressor, 1024, 0);
|
||||
blockCompressorStream.close();
|
||||
// check compressed output
|
||||
buf = bytesOut.toByteArray();
|
||||
int emSize = emptySize.get(compressor.getClass());
|
||||
Assert.assertEquals(
|
||||
joiner.join(name, "empty stream compressed output size != "
|
||||
+ emSize), emSize, buf.length);
|
||||
// use compressed output as input for decompression
|
||||
bytesIn = new ByteArrayInputStream(buf);
|
||||
// create decompression stream
|
||||
blockDecompressorStream = new BlockDecompressorStream(bytesIn,
|
||||
decompressor, 1024);
|
||||
// no byte is available because stream was closed
|
||||
assertEquals(joiner.join(name, " return value is not -1"), -1,
|
||||
blockDecompressorStream.read());
|
||||
} catch (IOException e) {
|
||||
fail(joiner.join(name, e.getMessage()));
|
||||
} finally {
|
||||
if (blockDecompressorStream != null)
|
||||
try {
|
||||
bytesOut.close();
|
||||
blockDecompressorStream.close();
|
||||
bytesIn.close();
|
||||
blockDecompressorStream.close();
|
||||
} catch (IOException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}),
|
||||
|
||||
COMPRESS_DECOMPRESS_BLOCK(new TesterCompressionStrategy() {
|
||||
private final Joiner joiner = Joiner.on("- ");
|
||||
private static final int BLOCK_SIZE = 512;
|
||||
private final byte[] operationBlock = new byte[BLOCK_SIZE];
|
||||
// Use default of 512 as bufferSize and compressionOverhead of
|
||||
// (1% of bufferSize + 12 bytes) = 18 bytes (zlib algorithm).
|
||||
private static final int overheadSpace = BLOCK_SIZE / 100 + 12;
|
||||
|
||||
@Override
|
||||
public void assertCompression(String name, Compressor compressor,
|
||||
Decompressor decompressor, byte[] originalRawData) {
|
||||
int off = 0;
|
||||
int len = originalRawData.length;
|
||||
int maxSize = BLOCK_SIZE - overheadSpace;
|
||||
int compresSize = 0;
|
||||
List<Integer> blockLabels = new ArrayList<Integer>();
|
||||
ByteArrayOutputStream compressedOut = new ByteArrayOutputStream();
|
||||
ByteArrayOutputStream decompressOut = new ByteArrayOutputStream();
|
||||
try {
|
||||
if (originalRawData.length > maxSize) {
|
||||
do {
|
||||
int bufLen = Math.min(len, maxSize);
|
||||
compressor.setInput(originalRawData, off, bufLen);
|
||||
compressor.finish();
|
||||
while (!compressor.finished()) {
|
||||
compresSize = compressor.compress(operationBlock, 0,
|
||||
operationBlock.length);
|
||||
compressedOut.write(operationBlock, 0, compresSize);
|
||||
blockLabels.add(compresSize);
|
||||
}
|
||||
compressor.reset();
|
||||
off += bufLen;
|
||||
len -= bufLen;
|
||||
} while (len > 0);
|
||||
}
|
||||
|
||||
off = 0;
|
||||
// compressed bytes
|
||||
byte[] compressedBytes = compressedOut.toByteArray();
|
||||
for (Integer step : blockLabels) {
|
||||
decompressor.setInput(compressedBytes, off, step);
|
||||
while (!decompressor.finished()) {
|
||||
int dSize = decompressor.decompress(operationBlock, 0,
|
||||
operationBlock.length);
|
||||
decompressOut.write(operationBlock, 0, dSize);
|
||||
}
|
||||
decompressor.reset();
|
||||
off = off + step;
|
||||
}
|
||||
assertArrayEquals(
|
||||
joiner.join(name, "byte arrays not equals error !!!"),
|
||||
originalRawData, decompressOut.toByteArray());
|
||||
} catch (Exception ex) {
|
||||
fail(joiner.join(name, ex.getMessage()));
|
||||
} finally {
|
||||
try {
|
||||
compressedOut.close();
|
||||
} catch (IOException e) {
|
||||
}
|
||||
try {
|
||||
decompressOut.close();
|
||||
} catch (IOException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
private final TesterCompressionStrategy testerStrategy;
|
||||
|
||||
CompressionTestStrategy(TesterCompressionStrategy testStrategy) {
|
||||
this.testerStrategy = testStrategy;
|
||||
}
|
||||
|
||||
public TesterCompressionStrategy getTesterStrategy() {
|
||||
return testerStrategy;
|
||||
}
|
||||
}
|
||||
|
||||
static final class TesterPair<T extends Compressor, E extends Decompressor> {
|
||||
private final T compressor;
|
||||
private final E decompressor;
|
||||
private final String name;
|
||||
|
||||
TesterPair(String name, T compressor, E decompressor) {
|
||||
this.compressor = compressor;
|
||||
this.decompressor = decompressor;
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
public void end() {
|
||||
Configuration cfg = new Configuration();
|
||||
compressor.reinit(cfg);
|
||||
compressor.end();
|
||||
decompressor.end();
|
||||
}
|
||||
|
||||
public T getCompressor() {
|
||||
return compressor;
|
||||
}
|
||||
|
||||
public E getDecompressor() {
|
||||
return decompressor;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Method for compressor availability check
|
||||
*/
|
||||
private static <T extends Compressor, E extends Decompressor> boolean isAvailable(TesterPair<T, E> pair) {
|
||||
Compressor compressor = pair.compressor;
|
||||
|
||||
if (compressor.getClass().isAssignableFrom(Lz4Compressor.class)
|
||||
&& (NativeCodeLoader.isNativeCodeLoaded()))
|
||||
return true;
|
||||
|
||||
else if (compressor.getClass().isAssignableFrom(BuiltInZlibDeflater.class)
|
||||
&& NativeCodeLoader.isNativeCodeLoaded())
|
||||
return true;
|
||||
|
||||
else if (compressor.getClass().isAssignableFrom(ZlibCompressor.class)) {
|
||||
return ZlibFactory.isNativeZlibLoaded(new Configuration());
|
||||
}
|
||||
else if (compressor.getClass().isAssignableFrom(SnappyCompressor.class)
|
||||
&& isNativeSnappyLoadable())
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
abstract static class TesterCompressionStrategy {
|
||||
|
||||
protected final Logger logger = Logger.getLogger(getClass());
|
||||
|
||||
abstract void assertCompression(String name, Compressor compressor,
|
||||
Decompressor decompressor, byte[] originalRawData);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,70 @@
|
|||
/**
|
||||
* 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.io.compress;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestCodecPool {
|
||||
private final String LEASE_COUNT_ERR =
|
||||
"Incorrect number of leased (de)compressors";
|
||||
DefaultCodec codec;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
this.codec = new DefaultCodec();
|
||||
this.codec.setConf(new Configuration());
|
||||
}
|
||||
|
||||
@Test(timeout = 1000)
|
||||
public void testCompressorPoolCounts() {
|
||||
// Get two compressors and return them
|
||||
Compressor comp1 = CodecPool.getCompressor(codec);
|
||||
Compressor comp2 = CodecPool.getCompressor(codec);
|
||||
assertEquals(LEASE_COUNT_ERR, 2,
|
||||
CodecPool.getLeasedCompressorsCount(codec));
|
||||
|
||||
CodecPool.returnCompressor(comp2);
|
||||
assertEquals(LEASE_COUNT_ERR, 1,
|
||||
CodecPool.getLeasedCompressorsCount(codec));
|
||||
|
||||
CodecPool.returnCompressor(comp1);
|
||||
assertEquals(LEASE_COUNT_ERR, 0,
|
||||
CodecPool.getLeasedCompressorsCount(codec));
|
||||
}
|
||||
|
||||
@Test(timeout = 1000)
|
||||
public void testDecompressorPoolCounts() {
|
||||
// Get two decompressors and return them
|
||||
Decompressor decomp1 = CodecPool.getDecompressor(codec);
|
||||
Decompressor decomp2 = CodecPool.getDecompressor(codec);
|
||||
assertEquals(LEASE_COUNT_ERR, 2,
|
||||
CodecPool.getLeasedDecompressorsCount(codec));
|
||||
|
||||
CodecPool.returnDecompressor(decomp2);
|
||||
assertEquals(LEASE_COUNT_ERR, 1,
|
||||
CodecPool.getLeasedDecompressorsCount(codec));
|
||||
|
||||
CodecPool.returnDecompressor(decomp1);
|
||||
assertEquals(LEASE_COUNT_ERR, 0,
|
||||
CodecPool.getLeasedDecompressorsCount(codec));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,101 @@
|
|||
/*
|
||||
* 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.io.compress;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
import java.util.Random;
|
||||
import org.apache.hadoop.io.compress.CompressDecompressTester.CompressionTestStrategy;
|
||||
import org.apache.hadoop.io.compress.lz4.Lz4Compressor;
|
||||
import org.apache.hadoop.io.compress.lz4.Lz4Decompressor;
|
||||
import org.apache.hadoop.io.compress.snappy.SnappyCompressor;
|
||||
import org.apache.hadoop.io.compress.snappy.SnappyDecompressor;
|
||||
import org.apache.hadoop.io.compress.zlib.BuiltInZlibDeflater;
|
||||
import org.apache.hadoop.io.compress.zlib.BuiltInZlibInflater;
|
||||
import org.junit.Test;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
||||
/**
|
||||
* Test for pairs:
|
||||
* <pre>
|
||||
* SnappyCompressor/SnappyDecompressor
|
||||
* Lz4Compressor/Lz4Decompressor
|
||||
* BuiltInZlibDeflater/new BuiltInZlibInflater
|
||||
*
|
||||
*
|
||||
* Note: we can't use ZlibCompressor/ZlibDecompressor here
|
||||
* because his constructor can throw exception (if native libraries not found)
|
||||
* For ZlibCompressor/ZlibDecompressor pair testing used {@code TestZlibCompressorDecompressor}
|
||||
*
|
||||
* </pre>
|
||||
*
|
||||
*/
|
||||
public class TestCompressorDecompressor {
|
||||
|
||||
private static final Random rnd = new Random(12345L);
|
||||
|
||||
@Test
|
||||
public void testCompressorDecompressor() {
|
||||
// no more for this data
|
||||
int SIZE = 44 * 1024;
|
||||
|
||||
byte[] rawData = generate(SIZE);
|
||||
try {
|
||||
CompressDecompressTester.of(rawData)
|
||||
.withCompressDecompressPair(new SnappyCompressor(), new SnappyDecompressor())
|
||||
.withCompressDecompressPair(new Lz4Compressor(), new Lz4Decompressor())
|
||||
.withCompressDecompressPair(new BuiltInZlibDeflater(), new BuiltInZlibInflater())
|
||||
.withTestCases(ImmutableSet.of(CompressionTestStrategy.COMPRESS_DECOMPRESS_SINGLE_BLOCK,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_BLOCK,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_ERRORS,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM))
|
||||
.test();
|
||||
|
||||
} catch (Exception ex) {
|
||||
fail("testCompressorDecompressor error !!!" + ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompressorDecompressorWithExeedBufferLimit() {
|
||||
int BYTE_SIZE = 100 * 1024;
|
||||
byte[] rawData = generate(BYTE_SIZE);
|
||||
try {
|
||||
CompressDecompressTester.of(rawData)
|
||||
.withCompressDecompressPair(
|
||||
new SnappyCompressor(BYTE_SIZE + BYTE_SIZE / 2),
|
||||
new SnappyDecompressor(BYTE_SIZE + BYTE_SIZE / 2))
|
||||
.withCompressDecompressPair(new Lz4Compressor(BYTE_SIZE),
|
||||
new Lz4Decompressor(BYTE_SIZE))
|
||||
.withTestCases(ImmutableSet.of(CompressionTestStrategy.COMPRESS_DECOMPRESS_SINGLE_BLOCK,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_BLOCK,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_ERRORS,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM))
|
||||
.test();
|
||||
|
||||
} catch (Exception ex) {
|
||||
fail("testCompressorDecompressorWithExeedBufferLimit error !!!" + ex);
|
||||
}
|
||||
}
|
||||
|
||||
public static byte[] generate(int size) {
|
||||
byte[] array = new byte[size];
|
||||
for (int i = 0; i < size; i++)
|
||||
array[i] = (byte) rnd.nextInt(16);
|
||||
return array;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,316 @@
|
|||
/**
|
||||
* 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.io.compress.lz4;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.DataOutputBuffer;
|
||||
import org.apache.hadoop.io.compress.BlockCompressorStream;
|
||||
import org.apache.hadoop.io.compress.BlockDecompressorStream;
|
||||
import org.apache.hadoop.io.compress.CompressionInputStream;
|
||||
import org.apache.hadoop.io.compress.CompressionOutputStream;
|
||||
import org.apache.hadoop.io.compress.Lz4Codec;
|
||||
import org.apache.hadoop.io.compress.lz4.Lz4Compressor;
|
||||
import org.apache.hadoop.io.compress.lz4.Lz4Decompressor;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assume.*;
|
||||
|
||||
public class TestLz4CompressorDecompressor {
|
||||
|
||||
private static final Random rnd = new Random(12345l);
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
assumeTrue(Lz4Codec.isNativeCodeLoaded());
|
||||
}
|
||||
|
||||
//test on NullPointerException in {@code compressor.setInput()}
|
||||
@Test
|
||||
public void testCompressorSetInputNullPointerException() {
|
||||
try {
|
||||
Lz4Compressor compressor = new Lz4Compressor();
|
||||
compressor.setInput(null, 0, 10);
|
||||
fail("testCompressorSetInputNullPointerException error !!!");
|
||||
} catch (NullPointerException ex) {
|
||||
// expected
|
||||
} catch (Exception e) {
|
||||
fail("testCompressorSetInputNullPointerException ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
//test on NullPointerException in {@code decompressor.setInput()}
|
||||
@Test
|
||||
public void testDecompressorSetInputNullPointerException() {
|
||||
try {
|
||||
Lz4Decompressor decompressor = new Lz4Decompressor();
|
||||
decompressor.setInput(null, 0, 10);
|
||||
fail("testDecompressorSetInputNullPointerException error !!!");
|
||||
} catch (NullPointerException ex) {
|
||||
// expected
|
||||
} catch (Exception e) {
|
||||
fail("testDecompressorSetInputNullPointerException ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
//test on ArrayIndexOutOfBoundsException in {@code compressor.setInput()}
|
||||
@Test
|
||||
public void testCompressorSetInputAIOBException() {
|
||||
try {
|
||||
Lz4Compressor compressor = new Lz4Compressor();
|
||||
compressor.setInput(new byte[] {}, -5, 10);
|
||||
fail("testCompressorSetInputAIOBException error !!!");
|
||||
} catch (ArrayIndexOutOfBoundsException ex) {
|
||||
// expected
|
||||
} catch (Exception ex) {
|
||||
fail("testCompressorSetInputAIOBException ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
//test on ArrayIndexOutOfBoundsException in {@code decompressor.setInput()}
|
||||
@Test
|
||||
public void testDecompressorSetInputAIOUBException() {
|
||||
try {
|
||||
Lz4Decompressor decompressor = new Lz4Decompressor();
|
||||
decompressor.setInput(new byte[] {}, -5, 10);
|
||||
fail("testDecompressorSetInputAIOBException error !!!");
|
||||
} catch (ArrayIndexOutOfBoundsException ex) {
|
||||
// expected
|
||||
} catch (Exception e) {
|
||||
fail("testDecompressorSetInputAIOBException ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
//test on NullPointerException in {@code compressor.compress()}
|
||||
@Test
|
||||
public void testCompressorCompressNullPointerException() {
|
||||
try {
|
||||
Lz4Compressor compressor = new Lz4Compressor();
|
||||
byte[] bytes = generate(1024 * 6);
|
||||
compressor.setInput(bytes, 0, bytes.length);
|
||||
compressor.compress(null, 0, 0);
|
||||
fail("testCompressorCompressNullPointerException error !!!");
|
||||
} catch (NullPointerException ex) {
|
||||
// expected
|
||||
} catch (Exception e) {
|
||||
fail("testCompressorCompressNullPointerException ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
//test on NullPointerException in {@code decompressor.decompress()}
|
||||
@Test
|
||||
public void testDecompressorCompressNullPointerException() {
|
||||
try {
|
||||
Lz4Decompressor decompressor = new Lz4Decompressor();
|
||||
byte[] bytes = generate(1024 * 6);
|
||||
decompressor.setInput(bytes, 0, bytes.length);
|
||||
decompressor.decompress(null, 0, 0);
|
||||
fail("testDecompressorCompressNullPointerException error !!!");
|
||||
} catch (NullPointerException ex) {
|
||||
// expected
|
||||
} catch (Exception e) {
|
||||
fail("testDecompressorCompressNullPointerException ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
//test on ArrayIndexOutOfBoundsException in {@code compressor.compress()}
|
||||
@Test
|
||||
public void testCompressorCompressAIOBException() {
|
||||
try {
|
||||
Lz4Compressor compressor = new Lz4Compressor();
|
||||
byte[] bytes = generate(1024 * 6);
|
||||
compressor.setInput(bytes, 0, bytes.length);
|
||||
compressor.compress(new byte[] {}, 0, -1);
|
||||
fail("testCompressorCompressAIOBException error !!!");
|
||||
} catch (ArrayIndexOutOfBoundsException ex) {
|
||||
// expected
|
||||
} catch (Exception e) {
|
||||
fail("testCompressorCompressAIOBException ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
//test on ArrayIndexOutOfBoundsException in decompressor.decompress()
|
||||
@Test
|
||||
public void testDecompressorCompressAIOBException() {
|
||||
try {
|
||||
Lz4Decompressor decompressor = new Lz4Decompressor();
|
||||
byte[] bytes = generate(1024 * 6);
|
||||
decompressor.setInput(bytes, 0, bytes.length);
|
||||
decompressor.decompress(new byte[] {}, 0, -1);
|
||||
fail("testDecompressorCompressAIOBException error !!!");
|
||||
} catch (ArrayIndexOutOfBoundsException ex) {
|
||||
// expected
|
||||
} catch (Exception e) {
|
||||
fail("testDecompressorCompressAIOBException ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
// test Lz4Compressor compressor.compress()
|
||||
@Test
|
||||
public void testSetInputWithBytesSizeMoreThenDefaultLz4CompressorByfferSize() {
|
||||
int BYTES_SIZE = 1024 * 64 + 1;
|
||||
try {
|
||||
Lz4Compressor compressor = new Lz4Compressor();
|
||||
byte[] bytes = generate(BYTES_SIZE);
|
||||
assertTrue("needsInput error !!!", compressor.needsInput());
|
||||
compressor.setInput(bytes, 0, bytes.length);
|
||||
byte[] emptyBytes = new byte[BYTES_SIZE];
|
||||
int csize = compressor.compress(emptyBytes, 0, bytes.length);
|
||||
assertTrue(
|
||||
"testSetInputWithBytesSizeMoreThenDefaultLz4CompressorByfferSize error !!!",
|
||||
csize != 0);
|
||||
} catch (Exception ex) {
|
||||
fail("testSetInputWithBytesSizeMoreThenDefaultLz4CompressorByfferSize ex error !!!");
|
||||
}
|
||||
}
|
||||
|
||||
// test compress/decompress process
|
||||
@Test
|
||||
public void testCompressDecompress() {
|
||||
int BYTE_SIZE = 1024 * 54;
|
||||
byte[] bytes = generate(BYTE_SIZE);
|
||||
Lz4Compressor compressor = new Lz4Compressor();
|
||||
try {
|
||||
compressor.setInput(bytes, 0, bytes.length);
|
||||
assertTrue("Lz4CompressDecompress getBytesRead error !!!",
|
||||
compressor.getBytesRead() > 0);
|
||||
assertTrue(
|
||||
"Lz4CompressDecompress getBytesWritten before compress error !!!",
|
||||
compressor.getBytesWritten() == 0);
|
||||
|
||||
byte[] compressed = new byte[BYTE_SIZE];
|
||||
int cSize = compressor.compress(compressed, 0, compressed.length);
|
||||
assertTrue(
|
||||
"Lz4CompressDecompress getBytesWritten after compress error !!!",
|
||||
compressor.getBytesWritten() > 0);
|
||||
Lz4Decompressor decompressor = new Lz4Decompressor();
|
||||
// set as input for decompressor only compressed data indicated with cSize
|
||||
decompressor.setInput(compressed, 0, cSize);
|
||||
byte[] decompressed = new byte[BYTE_SIZE];
|
||||
decompressor.decompress(decompressed, 0, decompressed.length);
|
||||
|
||||
assertTrue("testLz4CompressDecompress finished error !!!", decompressor.finished());
|
||||
assertArrayEquals(bytes, decompressed);
|
||||
compressor.reset();
|
||||
decompressor.reset();
|
||||
assertTrue("decompressor getRemaining error !!!",decompressor.getRemaining() == 0);
|
||||
} catch (Exception e) {
|
||||
fail("testLz4CompressDecompress ex error!!!");
|
||||
}
|
||||
}
|
||||
|
||||
// test compress/decompress with empty stream
|
||||
@Test
|
||||
public void testCompressorDecompressorEmptyStreamLogic() {
|
||||
ByteArrayInputStream bytesIn = null;
|
||||
ByteArrayOutputStream bytesOut = null;
|
||||
byte[] buf = null;
|
||||
BlockDecompressorStream blockDecompressorStream = null;
|
||||
try {
|
||||
// compress empty stream
|
||||
bytesOut = new ByteArrayOutputStream();
|
||||
BlockCompressorStream blockCompressorStream = new BlockCompressorStream(
|
||||
bytesOut, new Lz4Compressor(), 1024, 0);
|
||||
// close without write
|
||||
blockCompressorStream.close();
|
||||
// check compressed output
|
||||
buf = bytesOut.toByteArray();
|
||||
assertEquals("empty stream compressed output size != 4", 4, buf.length);
|
||||
// use compressed output as input for decompression
|
||||
bytesIn = new ByteArrayInputStream(buf);
|
||||
// create decompression stream
|
||||
blockDecompressorStream = new BlockDecompressorStream(bytesIn,
|
||||
new Lz4Decompressor(), 1024);
|
||||
// no byte is available because stream was closed
|
||||
assertEquals("return value is not -1", -1, blockDecompressorStream.read());
|
||||
} catch (Exception e) {
|
||||
fail("testCompressorDecompressorEmptyStreamLogic ex error !!!"
|
||||
+ e.getMessage());
|
||||
} finally {
|
||||
if (blockDecompressorStream != null)
|
||||
try {
|
||||
bytesIn.close();
|
||||
bytesOut.close();
|
||||
blockDecompressorStream.close();
|
||||
} catch (IOException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// test compress/decompress process through CompressionOutputStream/CompressionInputStream api
|
||||
@Test
|
||||
public void testCompressorDecopressorLogicWithCompressionStreams() {
|
||||
DataOutputStream deflateOut = null;
|
||||
DataInputStream inflateIn = null;
|
||||
int BYTE_SIZE = 1024 * 100;
|
||||
byte[] bytes = generate(BYTE_SIZE);
|
||||
int bufferSize = 262144;
|
||||
int compressionOverhead = (bufferSize / 6) + 32;
|
||||
try {
|
||||
DataOutputBuffer compressedDataBuffer = new DataOutputBuffer();
|
||||
CompressionOutputStream deflateFilter = new BlockCompressorStream(
|
||||
compressedDataBuffer, new Lz4Compressor(bufferSize), bufferSize,
|
||||
compressionOverhead);
|
||||
deflateOut = new DataOutputStream(new BufferedOutputStream(deflateFilter));
|
||||
deflateOut.write(bytes, 0, bytes.length);
|
||||
deflateOut.flush();
|
||||
deflateFilter.finish();
|
||||
|
||||
DataInputBuffer deCompressedDataBuffer = new DataInputBuffer();
|
||||
deCompressedDataBuffer.reset(compressedDataBuffer.getData(), 0,
|
||||
compressedDataBuffer.getLength());
|
||||
|
||||
CompressionInputStream inflateFilter = new BlockDecompressorStream(
|
||||
deCompressedDataBuffer, new Lz4Decompressor(bufferSize), bufferSize);
|
||||
|
||||
inflateIn = new DataInputStream(new BufferedInputStream(inflateFilter));
|
||||
|
||||
byte[] result = new byte[BYTE_SIZE];
|
||||
inflateIn.read(result);
|
||||
|
||||
assertArrayEquals("original array not equals compress/decompressed array", result,
|
||||
bytes);
|
||||
} catch (IOException e) {
|
||||
fail("testLz4CompressorDecopressorLogicWithCompressionStreams ex error !!!");
|
||||
} finally {
|
||||
try {
|
||||
if (deflateOut != null)
|
||||
deflateOut.close();
|
||||
if (inflateIn != null)
|
||||
inflateIn.close();
|
||||
} catch (Exception e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static byte[] generate(int size) {
|
||||
byte[] array = new byte[size];
|
||||
for (int i = 0; i < size; i++)
|
||||
array[i] = (byte)rnd.nextInt(16);
|
||||
return array;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,362 @@
|
|||
/**
|
||||
* 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.io.compress.zlib;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.junit.Assume.*;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.compress.CompressDecompressTester;
|
||||
import org.apache.hadoop.io.compress.Compressor;
|
||||
import org.apache.hadoop.io.compress.Decompressor;
|
||||
import org.apache.hadoop.io.compress.DecompressorStream;
|
||||
import org.apache.hadoop.io.compress.CompressDecompressTester.CompressionTestStrategy;
|
||||
import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
|
||||
import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
||||
public class TestZlibCompressorDecompressor {
|
||||
|
||||
private static final Random random = new Random(12345L);
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
assumeTrue(ZlibFactory.isNativeZlibLoaded(new Configuration()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZlibCompressorDecompressor() {
|
||||
try {
|
||||
int SIZE = 44 * 1024;
|
||||
byte[] rawData = generate(SIZE);
|
||||
|
||||
CompressDecompressTester.of(rawData)
|
||||
.withCompressDecompressPair(new ZlibCompressor(), new ZlibDecompressor())
|
||||
.withTestCases(ImmutableSet.of(CompressionTestStrategy.COMPRESS_DECOMPRESS_SINGLE_BLOCK,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_BLOCK,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_ERRORS,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM))
|
||||
.test();
|
||||
} catch (Exception ex) {
|
||||
fail("testCompressorDecompressor error !!!" + ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompressorDecompressorWithExeedBufferLimit() {
|
||||
int BYTE_SIZE = 100 * 1024;
|
||||
byte[] rawData = generate(BYTE_SIZE);
|
||||
try {
|
||||
CompressDecompressTester.of(rawData)
|
||||
.withCompressDecompressPair(
|
||||
new ZlibCompressor(
|
||||
org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel.BEST_COMPRESSION,
|
||||
CompressionStrategy.DEFAULT_STRATEGY,
|
||||
org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionHeader.DEFAULT_HEADER,
|
||||
BYTE_SIZE),
|
||||
new ZlibDecompressor(
|
||||
org.apache.hadoop.io.compress.zlib.ZlibDecompressor.CompressionHeader.DEFAULT_HEADER,
|
||||
BYTE_SIZE))
|
||||
.withTestCases(ImmutableSet.of(CompressionTestStrategy.COMPRESS_DECOMPRESS_SINGLE_BLOCK,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_BLOCK,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_ERRORS,
|
||||
CompressionTestStrategy.COMPRESS_DECOMPRESS_WITH_EMPTY_STREAM))
|
||||
.test();
|
||||
} catch (Exception ex) {
|
||||
fail("testCompressorDecompressorWithExeedBufferLimit error !!!" + ex);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testZlibCompressorDecompressorWithConfiguration() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, true);
|
||||
if (ZlibFactory.isNativeZlibLoaded(conf)) {
|
||||
byte[] rawData;
|
||||
int tryNumber = 5;
|
||||
int BYTE_SIZE = 10 * 1024;
|
||||
Compressor zlibCompressor = ZlibFactory.getZlibCompressor(conf);
|
||||
Decompressor zlibDecompressor = ZlibFactory.getZlibDecompressor(conf);
|
||||
rawData = generate(BYTE_SIZE);
|
||||
try {
|
||||
for (int i = 0; i < tryNumber; i++)
|
||||
compressDecompressZlib(rawData, (ZlibCompressor) zlibCompressor,
|
||||
(ZlibDecompressor) zlibDecompressor);
|
||||
zlibCompressor.reinit(conf);
|
||||
} catch (Exception ex) {
|
||||
fail("testZlibCompressorDecompressorWithConfiguration ex error " + ex);
|
||||
}
|
||||
} else {
|
||||
assertTrue("ZlibFactory is using native libs against request",
|
||||
ZlibFactory.isNativeZlibLoaded(conf));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZlibCompressDecompress() {
|
||||
byte[] rawData = null;
|
||||
int rawDataSize = 0;
|
||||
rawDataSize = 1024 * 64;
|
||||
rawData = generate(rawDataSize);
|
||||
try {
|
||||
ZlibCompressor compressor = new ZlibCompressor();
|
||||
ZlibDecompressor decompressor = new ZlibDecompressor();
|
||||
assertFalse("testZlibCompressDecompress finished error",
|
||||
compressor.finished());
|
||||
compressor.setInput(rawData, 0, rawData.length);
|
||||
assertTrue("testZlibCompressDecompress getBytesRead before error",
|
||||
compressor.getBytesRead() == 0);
|
||||
compressor.finish();
|
||||
|
||||
byte[] compressedResult = new byte[rawDataSize];
|
||||
int cSize = compressor.compress(compressedResult, 0, rawDataSize);
|
||||
assertTrue("testZlibCompressDecompress getBytesRead ather error",
|
||||
compressor.getBytesRead() == rawDataSize);
|
||||
assertTrue(
|
||||
"testZlibCompressDecompress compressed size no less then original size",
|
||||
cSize < rawDataSize);
|
||||
decompressor.setInput(compressedResult, 0, cSize);
|
||||
byte[] decompressedBytes = new byte[rawDataSize];
|
||||
decompressor.decompress(decompressedBytes, 0, decompressedBytes.length);
|
||||
assertArrayEquals("testZlibCompressDecompress arrays not equals ",
|
||||
rawData, decompressedBytes);
|
||||
compressor.reset();
|
||||
decompressor.reset();
|
||||
} catch (IOException ex) {
|
||||
fail("testZlibCompressDecompress ex !!!" + ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZlibCompressorDecompressorSetDictionary() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.setBoolean(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY, true);
|
||||
if (ZlibFactory.isNativeZlibLoaded(conf)) {
|
||||
Compressor zlibCompressor = ZlibFactory.getZlibCompressor(conf);
|
||||
Decompressor zlibDecompressor = ZlibFactory.getZlibDecompressor(conf);
|
||||
|
||||
checkSetDictionaryNullPointerException(zlibCompressor);
|
||||
checkSetDictionaryNullPointerException(zlibDecompressor);
|
||||
|
||||
checkSetDictionaryArrayIndexOutOfBoundsException(zlibDecompressor);
|
||||
checkSetDictionaryArrayIndexOutOfBoundsException(zlibCompressor);
|
||||
} else {
|
||||
assertTrue("ZlibFactory is using native libs against request",
|
||||
ZlibFactory.isNativeZlibLoaded(conf));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZlibFactory() {
|
||||
Configuration cfg = new Configuration();
|
||||
|
||||
assertTrue("testZlibFactory compression level error !!!",
|
||||
CompressionLevel.DEFAULT_COMPRESSION == ZlibFactory
|
||||
.getCompressionLevel(cfg));
|
||||
|
||||
assertTrue("testZlibFactory compression strategy error !!!",
|
||||
CompressionStrategy.DEFAULT_STRATEGY == ZlibFactory
|
||||
.getCompressionStrategy(cfg));
|
||||
|
||||
ZlibFactory.setCompressionLevel(cfg, CompressionLevel.BEST_COMPRESSION);
|
||||
assertTrue("testZlibFactory compression strategy error !!!",
|
||||
CompressionLevel.BEST_COMPRESSION == ZlibFactory
|
||||
.getCompressionLevel(cfg));
|
||||
|
||||
ZlibFactory.setCompressionStrategy(cfg, CompressionStrategy.FILTERED);
|
||||
assertTrue("testZlibFactory compression strategy error !!!",
|
||||
CompressionStrategy.FILTERED == ZlibFactory.getCompressionStrategy(cfg));
|
||||
}
|
||||
|
||||
|
||||
private boolean checkSetDictionaryNullPointerException(
|
||||
Decompressor decompressor) {
|
||||
try {
|
||||
decompressor.setDictionary(null, 0, 1);
|
||||
} catch (NullPointerException ex) {
|
||||
return true;
|
||||
} catch (Exception ex) {
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkSetDictionaryNullPointerException(Compressor compressor) {
|
||||
try {
|
||||
compressor.setDictionary(null, 0, 1);
|
||||
} catch (NullPointerException ex) {
|
||||
return true;
|
||||
} catch (Exception ex) {
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkSetDictionaryArrayIndexOutOfBoundsException(
|
||||
Compressor compressor) {
|
||||
try {
|
||||
compressor.setDictionary(new byte[] { (byte) 0 }, 0, -1);
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean checkSetDictionaryArrayIndexOutOfBoundsException(
|
||||
Decompressor decompressor) {
|
||||
try {
|
||||
decompressor.setDictionary(new byte[] { (byte) 0 }, 0, -1);
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private byte[] compressDecompressZlib(byte[] rawData,
|
||||
ZlibCompressor zlibCompressor, ZlibDecompressor zlibDecompressor)
|
||||
throws IOException {
|
||||
int cSize = 0;
|
||||
byte[] compressedByte = new byte[rawData.length];
|
||||
byte[] decompressedRawData = new byte[rawData.length];
|
||||
zlibCompressor.setInput(rawData, 0, rawData.length);
|
||||
zlibCompressor.finish();
|
||||
while (!zlibCompressor.finished()) {
|
||||
cSize = zlibCompressor.compress(compressedByte, 0, compressedByte.length);
|
||||
}
|
||||
zlibCompressor.reset();
|
||||
|
||||
assertTrue(zlibDecompressor.getBytesWritten() == 0);
|
||||
assertTrue(zlibDecompressor.getBytesRead() == 0);
|
||||
assertTrue(zlibDecompressor.needsInput());
|
||||
zlibDecompressor.setInput(compressedByte, 0, cSize);
|
||||
assertFalse(zlibDecompressor.needsInput());
|
||||
while (!zlibDecompressor.finished()) {
|
||||
zlibDecompressor.decompress(decompressedRawData, 0,
|
||||
decompressedRawData.length);
|
||||
}
|
||||
assertTrue(zlibDecompressor.getBytesWritten() == rawData.length);
|
||||
assertTrue(zlibDecompressor.getBytesRead() == cSize);
|
||||
zlibDecompressor.reset();
|
||||
assertTrue(zlibDecompressor.getRemaining() == 0);
|
||||
assertArrayEquals(
|
||||
"testZlibCompressorDecompressorWithConfiguration array equals error",
|
||||
rawData, decompressedRawData);
|
||||
|
||||
return decompressedRawData;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuiltInGzipDecompressorExceptions() {
|
||||
BuiltInGzipDecompressor decompresser = new BuiltInGzipDecompressor();
|
||||
try {
|
||||
decompresser.setInput(null, 0, 1);
|
||||
} catch (NullPointerException ex) {
|
||||
// expected
|
||||
} catch (Exception ex) {
|
||||
fail("testBuiltInGzipDecompressorExceptions npe error " + ex);
|
||||
}
|
||||
|
||||
try {
|
||||
decompresser.setInput(new byte[] { 0 }, 0, -1);
|
||||
} catch (ArrayIndexOutOfBoundsException ex) {
|
||||
// expected
|
||||
} catch (Exception ex) {
|
||||
fail("testBuiltInGzipDecompressorExceptions aioob error" + ex);
|
||||
}
|
||||
|
||||
assertTrue("decompresser.getBytesRead error",
|
||||
decompresser.getBytesRead() == 0);
|
||||
assertTrue("decompresser.getRemaining error",
|
||||
decompresser.getRemaining() == 0);
|
||||
decompresser.reset();
|
||||
decompresser.end();
|
||||
|
||||
InputStream decompStream = null;
|
||||
try {
|
||||
// invalid 0 and 1 bytes , must be 31, -117
|
||||
int buffSize = 1 * 1024;
|
||||
byte buffer[] = new byte[buffSize];
|
||||
Decompressor decompressor = new BuiltInGzipDecompressor();
|
||||
DataInputBuffer gzbuf = new DataInputBuffer();
|
||||
decompStream = new DecompressorStream(gzbuf, decompressor);
|
||||
gzbuf.reset(new byte[] { 0, 0, 1, 1, 1, 1, 11, 1, 1, 1, 1 }, 11);
|
||||
decompStream.read(buffer);
|
||||
} catch (IOException ioex) {
|
||||
// expected
|
||||
} catch (Exception ex) {
|
||||
fail("invalid 0 and 1 byte in gzip stream" + ex);
|
||||
}
|
||||
|
||||
// invalid 2 byte, must be 8
|
||||
try {
|
||||
int buffSize = 1 * 1024;
|
||||
byte buffer[] = new byte[buffSize];
|
||||
Decompressor decompressor = new BuiltInGzipDecompressor();
|
||||
DataInputBuffer gzbuf = new DataInputBuffer();
|
||||
decompStream = new DecompressorStream(gzbuf, decompressor);
|
||||
gzbuf.reset(new byte[] { 31, -117, 7, 1, 1, 1, 1, 11, 1, 1, 1, 1 }, 11);
|
||||
decompStream.read(buffer);
|
||||
} catch (IOException ioex) {
|
||||
// expected
|
||||
} catch (Exception ex) {
|
||||
fail("invalid 2 byte in gzip stream" + ex);
|
||||
}
|
||||
|
||||
try {
|
||||
int buffSize = 1 * 1024;
|
||||
byte buffer[] = new byte[buffSize];
|
||||
Decompressor decompressor = new BuiltInGzipDecompressor();
|
||||
DataInputBuffer gzbuf = new DataInputBuffer();
|
||||
decompStream = new DecompressorStream(gzbuf, decompressor);
|
||||
gzbuf.reset(new byte[] { 31, -117, 8, -32, 1, 1, 1, 11, 1, 1, 1, 1 }, 11);
|
||||
decompStream.read(buffer);
|
||||
} catch (IOException ioex) {
|
||||
// expected
|
||||
} catch (Exception ex) {
|
||||
fail("invalid 3 byte in gzip stream" + ex);
|
||||
}
|
||||
try {
|
||||
int buffSize = 1 * 1024;
|
||||
byte buffer[] = new byte[buffSize];
|
||||
Decompressor decompressor = new BuiltInGzipDecompressor();
|
||||
DataInputBuffer gzbuf = new DataInputBuffer();
|
||||
decompStream = new DecompressorStream(gzbuf, decompressor);
|
||||
gzbuf.reset(new byte[] { 31, -117, 8, 4, 1, 1, 1, 11, 1, 1, 1, 1 }, 11);
|
||||
decompStream.read(buffer);
|
||||
} catch (IOException ioex) {
|
||||
// expected
|
||||
} catch (Exception ex) {
|
||||
fail("invalid 3 byte make hasExtraField" + ex);
|
||||
}
|
||||
}
|
||||
|
||||
public static byte[] generate(int size) {
|
||||
byte[] data = new byte[size];
|
||||
for (int i = 0; i < size; i++)
|
||||
data[i] = (byte)random.nextInt(16);
|
||||
return data;
|
||||
}
|
||||
}
|
|
@ -446,7 +446,13 @@ public class TestNativeIO {
|
|||
NativeIO.renameTo(nonExistentFile, targetFile);
|
||||
Assert.fail();
|
||||
} catch (NativeIOException e) {
|
||||
Assert.assertEquals(e.getErrno(), Errno.ENOENT);
|
||||
if (Path.WINDOWS) {
|
||||
Assert.assertEquals(
|
||||
String.format("The system cannot find the file specified.%n"),
|
||||
e.getMessage());
|
||||
} else {
|
||||
Assert.assertEquals(Errno.ENOENT, e.getErrno());
|
||||
}
|
||||
}
|
||||
|
||||
// Test renaming a file to itself. It should succeed and do nothing.
|
||||
|
@ -465,7 +471,13 @@ public class TestNativeIO {
|
|||
NativeIO.renameTo(sourceFile, badTarget);
|
||||
Assert.fail();
|
||||
} catch (NativeIOException e) {
|
||||
Assert.assertEquals(e.getErrno(), Errno.ENOTDIR);
|
||||
if (Path.WINDOWS) {
|
||||
Assert.assertEquals(
|
||||
String.format("The parameter is incorrect.%n"),
|
||||
e.getMessage());
|
||||
} else {
|
||||
Assert.assertEquals(Errno.ENOTDIR, e.getErrno());
|
||||
}
|
||||
}
|
||||
|
||||
FileUtils.deleteQuietly(TEST_DIR);
|
||||
|
|
|
@ -24,7 +24,9 @@ import java.util.List;
|
|||
import org.apache.commons.configuration.SubsetConfiguration;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.*;
|
||||
|
||||
import org.apache.hadoop.metrics2.MetricsRecord;
|
||||
import org.apache.hadoop.metrics2.MetricsTag;
|
||||
import org.apache.hadoop.metrics2.impl.ConfigBuilder;
|
||||
import static org.apache.hadoop.metrics2.lib.Interns.*;
|
||||
|
@ -38,6 +40,8 @@ public class TestPatternFilter {
|
|||
SubsetConfiguration empty = new ConfigBuilder().subset("");
|
||||
shouldAccept(empty, "anything");
|
||||
shouldAccept(empty, Arrays.asList(tag("key", "desc", "value")));
|
||||
shouldAccept(empty, mockMetricsRecord("anything", Arrays.asList(
|
||||
tag("key", "desc", "value"))));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -50,9 +54,15 @@ public class TestPatternFilter {
|
|||
shouldAccept(wl, "foo");
|
||||
shouldAccept(wl, Arrays.asList(tag("bar", "", ""),
|
||||
tag("foo", "", "f")));
|
||||
shouldAccept(wl, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("bar", "", ""), tag("foo", "", "f"))));
|
||||
shouldReject(wl, "bar");
|
||||
shouldReject(wl, Arrays.asList(tag("bar", "", "")));
|
||||
shouldReject(wl, Arrays.asList(tag("foo", "", "boo")));
|
||||
shouldReject(wl, mockMetricsRecord("bar", Arrays.asList(
|
||||
tag("foo", "", "f"))));
|
||||
shouldReject(wl, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("bar", "", ""))));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -64,9 +74,15 @@ public class TestPatternFilter {
|
|||
.add("p.exclude.tags", "foo:f").subset("p");
|
||||
shouldAccept(bl, "bar");
|
||||
shouldAccept(bl, Arrays.asList(tag("bar", "", "")));
|
||||
shouldAccept(bl, mockMetricsRecord("bar", Arrays.asList(
|
||||
tag("bar", "", ""))));
|
||||
shouldReject(bl, "foo");
|
||||
shouldReject(bl, Arrays.asList(tag("bar", "", ""),
|
||||
tag("foo", "", "f")));
|
||||
shouldReject(bl, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("bar", "", ""))));
|
||||
shouldReject(bl, mockMetricsRecord("bar", Arrays.asList(
|
||||
tag("bar", "", ""), tag("foo", "", "f"))));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -81,10 +97,18 @@ public class TestPatternFilter {
|
|||
.add("p.exclude.tags", "bar:b").subset("p");
|
||||
shouldAccept(c, "foo");
|
||||
shouldAccept(c, Arrays.asList(tag("foo", "", "f")));
|
||||
shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("foo", "", "f"))));
|
||||
shouldReject(c, "bar");
|
||||
shouldReject(c, Arrays.asList(tag("bar", "", "b")));
|
||||
shouldReject(c, mockMetricsRecord("bar", Arrays.asList(
|
||||
tag("foo", "", "f"))));
|
||||
shouldReject(c, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("bar", "", "b"))));
|
||||
shouldAccept(c, "foobar");
|
||||
shouldAccept(c, Arrays.asList(tag("foobar", "", "")));
|
||||
shouldAccept(c, mockMetricsRecord("foobar", Arrays.asList(
|
||||
tag("foobar", "", ""))));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -98,6 +122,8 @@ public class TestPatternFilter {
|
|||
.add("p.exclude.tags", "foo:f").subset("p");
|
||||
shouldAccept(c, "foo");
|
||||
shouldAccept(c, Arrays.asList(tag("foo", "", "f")));
|
||||
shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("foo", "", "f"))));
|
||||
}
|
||||
|
||||
static void shouldAccept(SubsetConfiguration conf, String s) {
|
||||
|
@ -110,6 +136,17 @@ public class TestPatternFilter {
|
|||
assertTrue("accepts "+ tags, newRegexFilter(conf).accepts(tags));
|
||||
}
|
||||
|
||||
/**
|
||||
* Asserts that filters with the given configuration accept the given record.
|
||||
*
|
||||
* @param conf SubsetConfiguration containing filter configuration
|
||||
* @param record MetricsRecord to check
|
||||
*/
|
||||
static void shouldAccept(SubsetConfiguration conf, MetricsRecord record) {
|
||||
assertTrue("accepts " + record, newGlobFilter(conf).accepts(record));
|
||||
assertTrue("accepts " + record, newRegexFilter(conf).accepts(record));
|
||||
}
|
||||
|
||||
static void shouldReject(SubsetConfiguration conf, String s) {
|
||||
assertTrue("rejects "+ s, !newGlobFilter(conf).accepts(s));
|
||||
assertTrue("rejects "+ s, !newRegexFilter(conf).accepts(s));
|
||||
|
@ -120,6 +157,17 @@ public class TestPatternFilter {
|
|||
assertTrue("rejects "+ tags, !newRegexFilter(conf).accepts(tags));
|
||||
}
|
||||
|
||||
/**
|
||||
* Asserts that filters with the given configuration reject the given record.
|
||||
*
|
||||
* @param conf SubsetConfiguration containing filter configuration
|
||||
* @param record MetricsRecord to check
|
||||
*/
|
||||
static void shouldReject(SubsetConfiguration conf, MetricsRecord record) {
|
||||
assertTrue("rejects " + record, !newGlobFilter(conf).accepts(record));
|
||||
assertTrue("rejects " + record, !newRegexFilter(conf).accepts(record));
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new glob filter with a config object
|
||||
* @param conf the config object
|
||||
|
@ -141,4 +189,19 @@ public class TestPatternFilter {
|
|||
f.init(conf);
|
||||
return f;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a mock MetricsRecord with the given name and tags.
|
||||
*
|
||||
* @param name String name
|
||||
* @param tags List<MetricsTag> tags
|
||||
* @return MetricsRecord newly created mock
|
||||
*/
|
||||
private static MetricsRecord mockMetricsRecord(String name,
|
||||
List<MetricsTag> tags) {
|
||||
MetricsRecord record = mock(MetricsRecord.class);
|
||||
when(record.name()).thenReturn(name);
|
||||
when(record.tags()).thenReturn(tags);
|
||||
return record;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,31 +20,32 @@ package org.apache.hadoop.net;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestNetworkTopologyWithNodeGroup {
|
||||
private final static NetworkTopologyWithNodeGroup cluster = new
|
||||
NetworkTopologyWithNodeGroup();
|
||||
|
||||
private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
|
||||
DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1/s1"),
|
||||
DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1/s1"),
|
||||
DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r1/s2"),
|
||||
DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2/s3"),
|
||||
DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2/s3"),
|
||||
DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d1/r2/s4"),
|
||||
DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3/s5"),
|
||||
DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3/s6")
|
||||
private final static NodeBase dataNodes[] = new NodeBase[] {
|
||||
new NodeBase("h1", "/d1/r1/s1"),
|
||||
new NodeBase("h2", "/d1/r1/s1"),
|
||||
new NodeBase("h3", "/d1/r1/s2"),
|
||||
new NodeBase("h4", "/d1/r2/s3"),
|
||||
new NodeBase("h5", "/d1/r2/s3"),
|
||||
new NodeBase("h6", "/d1/r2/s4"),
|
||||
new NodeBase("h7", "/d2/r3/s5"),
|
||||
new NodeBase("h8", "/d2/r3/s6")
|
||||
};
|
||||
|
||||
private final static NodeBase computeNode = new NodeBase("/d1/r1/s1/h9");
|
||||
|
||||
private final static NodeBase rackOnlyNode = new NodeBase("h10", "/r2");
|
||||
|
||||
static {
|
||||
for(int i=0; i<dataNodes.length; i++) {
|
||||
cluster.add(dataNodes[i]);
|
||||
|
@ -96,7 +97,7 @@ public class TestNetworkTopologyWithNodeGroup {
|
|||
|
||||
@Test
|
||||
public void testPseudoSortByDistance() throws Exception {
|
||||
DatanodeDescriptor[] testNodes = new DatanodeDescriptor[4];
|
||||
NodeBase[] testNodes = new NodeBase[4];
|
||||
|
||||
// array contains both local node, local node group & local rack node
|
||||
testNodes[0] = dataNodes[1];
|
||||
|
@ -147,7 +148,7 @@ public class TestNetworkTopologyWithNodeGroup {
|
|||
private Map<Node, Integer> pickNodesAtRandom(int numNodes,
|
||||
String excludedScope) {
|
||||
Map<Node, Integer> frequency = new HashMap<Node, Integer>();
|
||||
for (DatanodeDescriptor dnd : dataNodes) {
|
||||
for (NodeBase dnd : dataNodes) {
|
||||
frequency.put(dnd, 0);
|
||||
}
|
||||
|
||||
|
@ -161,6 +162,12 @@ public class TestNetworkTopologyWithNodeGroup {
|
|||
/**
|
||||
* This test checks that chooseRandom works for an excluded node.
|
||||
*/
|
||||
/**
|
||||
* Test replica placement policy in case last node is invalid.
|
||||
* We create 6 nodes but the last node is in fault topology (with rack info),
|
||||
* so cannot be added to cluster. We should test proper exception is thrown in
|
||||
* adding node but shouldn't affect the cluster.
|
||||
*/
|
||||
@Test
|
||||
public void testChooseRandomExcludedNode() {
|
||||
String scope = "~" + NodeBase.getPath(dataNodes[0]);
|
||||
|
@ -172,4 +179,22 @@ public class TestNetworkTopologyWithNodeGroup {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This test checks that adding a node with invalid topology will be failed
|
||||
* with an exception to show topology is invalid.
|
||||
*/
|
||||
@Test
|
||||
public void testAddNodeWithInvalidTopology() {
|
||||
// The last node is a node with invalid topology
|
||||
try {
|
||||
cluster.add(rackOnlyNode);
|
||||
fail("Exception should be thrown, so we should not have reached here.");
|
||||
} catch (Exception e) {
|
||||
if (!(e instanceof IllegalArgumentException)) {
|
||||
fail("Expecting IllegalArgumentException, but caught:" + e);
|
||||
}
|
||||
assertTrue(e.getMessage().contains("illegal network location"));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -20,6 +20,9 @@ package org.apache.hadoop.test;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.lang.management.ThreadInfo;
|
||||
import java.lang.management.ThreadMXBean;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Random;
|
||||
|
@ -330,4 +333,33 @@ public abstract class GenericTestUtils {
|
|||
" but got:\n" + output,
|
||||
Pattern.compile(pattern).matcher(output).find());
|
||||
}
|
||||
|
||||
public static void assertValueNear(long expected, long actual, long allowedError) {
|
||||
assertValueWithinRange(expected - allowedError, expected + allowedError, actual);
|
||||
}
|
||||
|
||||
public static void assertValueWithinRange(long expectedMin, long expectedMax,
|
||||
long actual) {
|
||||
Assert.assertTrue("Expected " + actual + " to be in range (" + expectedMin + ","
|
||||
+ expectedMax + ")", expectedMin <= actual && actual <= expectedMax);
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert that there are no threads running whose name matches the
|
||||
* given regular expression.
|
||||
* @param regex the regex to match against
|
||||
*/
|
||||
public static void assertNoThreadsMatching(String regex) {
|
||||
Pattern pattern = Pattern.compile(regex);
|
||||
ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
|
||||
|
||||
ThreadInfo[] infos = threadBean.getThreadInfo(threadBean.getAllThreadIds(), 20);
|
||||
for (ThreadInfo info : infos) {
|
||||
if (info == null) continue;
|
||||
if (pattern.matcher(info.getThreadName()).matches()) {
|
||||
Assert.fail("Leaked thread: " + info + "\n" +
|
||||
Joiner.on("\n").join(info.getStackTrace()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,9 @@ package org.apache.hadoop.test;
|
|||
import java.io.Closeable;
|
||||
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
import org.mockito.stubbing.Stubber;
|
||||
|
||||
public abstract class MockitoUtil {
|
||||
|
||||
|
@ -33,4 +36,29 @@ public abstract class MockitoUtil {
|
|||
return Mockito.mock(clazz,
|
||||
Mockito.withSettings().extraInterfaces(Closeable.class));
|
||||
}
|
||||
|
||||
/**
|
||||
* Throw an exception from the mock/spy only in the case that the
|
||||
* call stack at the time the method has a line which matches the given
|
||||
* pattern.
|
||||
*
|
||||
* @param t the Throwable to throw
|
||||
* @param pattern the pattern against which to match the call stack trace
|
||||
* @return the stub in progress
|
||||
*/
|
||||
public static Stubber doThrowWhenCallStackMatches(
|
||||
final Throwable t, final String pattern) {
|
||||
return Mockito.doAnswer(new Answer<Object>() {
|
||||
@Override
|
||||
public Object answer(InvocationOnMock invocation) throws Throwable {
|
||||
t.setStackTrace(Thread.currentThread().getStackTrace());
|
||||
for (StackTraceElement elem : t.getStackTrace()) {
|
||||
if (elem.toString().matches(pattern)) {
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
return invocation.callRealMethod();
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,9 @@ Trunk (Unreleased)
|
|||
|
||||
IMPROVEMENTS
|
||||
|
||||
HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
|
||||
(Junping Du via llu)
|
||||
|
||||
HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
|
||||
HdfsConstants. (Harsh J Chouraria via atm)
|
||||
|
||||
|
@ -315,6 +318,9 @@ Trunk (Unreleased)
|
|||
HDFS-4586. TestDataDirs.testGetDataDirsFromURIs fails with all directories
|
||||
in dfs.datanode.data.dir are invalid. (Ivan Mitic via atm)
|
||||
|
||||
HDFS-4646. createNNProxyWithClientProtocol ignores configured timeout
|
||||
value (Jagane Sundar via cos)
|
||||
|
||||
BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
|
||||
|
||||
HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
|
||||
|
@ -343,12 +349,21 @@ Trunk (Unreleased)
|
|||
|
||||
HDFS-4604. TestJournalNode fails on Windows. (Ivan Mitic via suresh)
|
||||
|
||||
HDFS-4625. Make TestNNWithQJM#testNewNamenodeTakesOverWriter work on
|
||||
Windows. (Ivan Mitic via suresh)
|
||||
|
||||
HDFS-4674. TestBPOfferService fails on Windows due to failure parsing
|
||||
datanode data directory as URI. (Chris Nauroth via suresh)
|
||||
|
||||
Release 2.0.5-beta - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
HDFS-1804. Add a new block-volume device choosing policy that looks at
|
||||
free space. (atm)
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
HDFS-4222. NN is unresponsive and loses heartbeats from DNs when
|
||||
|
@ -378,6 +393,15 @@ Release 2.0.5-beta - UNRELEASED
|
|||
|
||||
HDFS-4618. Default transaction interval for checkpoints is too low. (todd)
|
||||
|
||||
HDFS-4525. Provide an API for knowing that whether file is closed or not.
|
||||
(SreeHari via umamahesh)
|
||||
|
||||
HDFS-3940. Add Gset#clear method and clear the block map when namenode is
|
||||
shutdown. (suresh)
|
||||
|
||||
HDFS-4679. Namenode operation checks should be done in a consistent
|
||||
manner. (suresh)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -460,6 +484,25 @@ Release 2.0.5-beta - UNRELEASED
|
|||
HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
|
||||
(szetszwo)
|
||||
|
||||
HDFS-4655. DNA_FINALIZE is logged as being an unknown command by the DN
|
||||
when received from the standby NN. (atm)
|
||||
|
||||
HDFS-4656. DN heartbeat loop can be briefly tight. (atm)
|
||||
|
||||
HDFS-4658. Standby NN will log that it has received a block report "after
|
||||
becoming active" (atm)
|
||||
|
||||
HDFS-3981. Fix handling of FSN lock in getBlockLocations. (Xiaobo Peng
|
||||
and todd via todd)
|
||||
|
||||
HDFS-4676. TestHDFSFileSystemContract should set MiniDFSCluster variable
|
||||
to null to free up memory. (suresh)
|
||||
|
||||
HDFS-4669. TestBlockPoolManager fails using IBM java. (Tian Hong Wang via
|
||||
suresh)
|
||||
|
||||
HDFS-4643. Fix flakiness in TestQuorumJournalManager. (todd)
|
||||
|
||||
Release 2.0.4-alpha - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -2433,6 +2476,20 @@ Release 2.0.0-alpha - 05-23-2012
|
|||
|
||||
HDFS-3039. Address findbugs and javadoc warnings on branch. (todd via atm)
|
||||
|
||||
Release 0.23.8 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
||||
HDFS-4477. Secondary namenode may retain old tokens (daryn via kihwal)
|
||||
|
||||
Release 0.23.7 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -2477,6 +2534,10 @@ Release 0.23.7 - UNRELEASED
|
|||
HDFS-4581. checkDiskError should not be called on network errors (Rohit
|
||||
Kochar via kihwal)
|
||||
|
||||
HDFS-4649. Webhdfs cannot list large directories (daryn via kihwal)
|
||||
|
||||
HDFS-4548. Webhdfs doesn't renegotiate SPNEGO token (daryn via kihwal)
|
||||
|
||||
Release 0.23.6 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -1547,6 +1547,21 @@ public class DFSClient implements java.io.Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close status of a file
|
||||
* @return true if file is already closed
|
||||
*/
|
||||
public boolean isFileClosed(String src) throws IOException{
|
||||
checkOpen();
|
||||
try {
|
||||
return namenode.isFileClosed(src);
|
||||
} catch(RemoteException re) {
|
||||
throw re.unwrapRemoteException(AccessControlException.class,
|
||||
FileNotFoundException.class,
|
||||
UnresolvedPathException.class);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the file info for a specific file or directory. If src
|
||||
* refers to a symlink then the FileStatus of the link is returned.
|
||||
|
|
|
@ -377,6 +377,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final String DFS_DATANODE_PLUGINS_KEY = "dfs.datanode.plugins";
|
||||
public static final String DFS_DATANODE_FSDATASET_FACTORY_KEY = "dfs.datanode.fsdataset.factory";
|
||||
public static final String DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_POLICY_KEY = "dfs.datanode.fsdataset.volume.choosing.policy";
|
||||
public static final String DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY = "dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold";
|
||||
public static final long DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_DEFAULT = 1024L * 1024L * 1024L * 10L; // 10 GB
|
||||
public static final String DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY = "dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-percent";
|
||||
public static final float DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_DEFAULT = 0.75f;
|
||||
public static final String DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY = "dfs.datanode.socket.write.timeout";
|
||||
public static final String DFS_DATANODE_STARTUP_KEY = "dfs.datanode.startup";
|
||||
public static final String DFS_NAMENODE_PLUGINS_KEY = "dfs.namenode.plugins";
|
||||
|
|
|
@ -460,7 +460,7 @@ public class DFSUtil {
|
|||
|
||||
// Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
|
||||
// across all of the configured nameservices and namenodes.
|
||||
Map<String, Map<String, InetSocketAddress>> ret = Maps.newHashMap();
|
||||
Map<String, Map<String, InetSocketAddress>> ret = Maps.newLinkedHashMap();
|
||||
for (String nsId : emptyAsSingletonNull(nameserviceIds)) {
|
||||
Map<String, InetSocketAddress> isas =
|
||||
getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
|
||||
|
|
|
@ -917,4 +917,17 @@ public class DistributedFileSystem extends FileSystem {
|
|||
public boolean isInSafeMode() throws IOException {
|
||||
return setSafeMode(SafeModeAction.SAFEMODE_GET, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the close status of a file
|
||||
* @param src The path to the file
|
||||
*
|
||||
* @return return true if file is closed
|
||||
* @throws FileNotFoundException if the file does not exist.
|
||||
* @throws IOException If an I/O error occurred
|
||||
*/
|
||||
public boolean isFileClosed(Path src) throws IOException {
|
||||
return dfs.isFileClosed(getPathName(src));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -260,7 +260,9 @@ public class NameNodeProxies {
|
|||
final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
|
||||
ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
|
||||
ClientNamenodeProtocolPB.class, version, address, ugi, conf,
|
||||
NetUtils.getDefaultSocketFactory(conf), 0, defaultPolicy).getProxy();
|
||||
NetUtils.getDefaultSocketFactory(conf),
|
||||
org.apache.hadoop.ipc.Client.getTimeout(conf), defaultPolicy)
|
||||
.getProxy();
|
||||
|
||||
if (withRetries) { // create the proxy with retries
|
||||
|
||||
|
|
|
@ -758,6 +758,20 @@ public interface ClientProtocol {
|
|||
public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
|
||||
FileNotFoundException, UnresolvedLinkException, IOException;
|
||||
|
||||
/**
|
||||
* Get the close status of a file
|
||||
* @param src The string representation of the path to the file
|
||||
*
|
||||
* @return return true if file is closed
|
||||
* @throws AccessControlException permission denied
|
||||
* @throws FileNotFoundException file <code>src</code> is not found
|
||||
* @throws UnresolvedLinkException if the path contains a symlink.
|
||||
* @throws IOException If an I/O error occurred
|
||||
*/
|
||||
@Idempotent
|
||||
public boolean isFileClosed(String src) throws AccessControlException,
|
||||
FileNotFoundException, UnresolvedLinkException, IOException;
|
||||
|
||||
/**
|
||||
* Get the file info for a specific file or directory. If the path
|
||||
* refers to a symlink then the FileStatus of the symlink is returned.
|
||||
|
|
|
@ -96,7 +96,8 @@ public class LayoutVersion {
|
|||
OPTIMIZE_PERSIST_BLOCKS(-40,
|
||||
"Serialize block lists with delta-encoded variable length ints, " +
|
||||
"add OP_UPDATE_BLOCKS"),
|
||||
RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT);
|
||||
RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT),
|
||||
ADD_INODE_ID(-42, -40, "Assign a unique inode id for each inode", false);
|
||||
|
||||
final int lv;
|
||||
final int ancestorLV;
|
||||
|
|
|
@ -76,6 +76,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPre
|
|||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
|
||||
|
@ -864,4 +866,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public IsFileClosedResponseProto isFileClosed(
|
||||
RpcController controller, IsFileClosedRequestProto request)
|
||||
throws ServiceException {
|
||||
try {
|
||||
boolean result = server.isFileClosed(request.getSrc());
|
||||
return IsFileClosedResponseProto.newBuilder().setResult(result).build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -77,6 +77,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLis
|
|||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
|
||||
|
@ -851,6 +852,19 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean isFileClosed(String src) throws AccessControlException,
|
||||
FileNotFoundException, UnresolvedLinkException, IOException {
|
||||
IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
|
||||
.setSrc(src).build();
|
||||
try {
|
||||
return rpcProxy.isFileClosed(null, req).getResult();
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getUnderlyingProxyObject() {
|
||||
return rpcProxy;
|
||||
|
|
|
@ -311,6 +311,23 @@ public class DelegationTokenSecretManager
|
|||
}
|
||||
}
|
||||
|
||||
@Override //AbstractDelegationTokenManager
|
||||
protected void logExpireToken(final DelegationTokenIdentifier dtId)
|
||||
throws IOException {
|
||||
synchronized (noInterruptsLock) {
|
||||
// The edit logging code will fail catastrophically if it
|
||||
// is interrupted during a logSync, since the interrupt
|
||||
// closes the edit log files. Doing this inside the
|
||||
// above lock and then checking interruption status
|
||||
// prevents this bug.
|
||||
if (Thread.interrupted()) {
|
||||
throw new InterruptedIOException(
|
||||
"Interrupted before expiring delegation token");
|
||||
}
|
||||
namesystem.logExpireDelegationToken(dtId);
|
||||
}
|
||||
}
|
||||
|
||||
/** A utility method for creating credentials. */
|
||||
public static Credentials createCredentials(final NameNode namenode,
|
||||
final UserGroupInformation ugi, final String renewer) throws IOException {
|
||||
|
|
|
@ -1569,8 +1569,8 @@ public class BlockManager {
|
|||
node.receivedBlockReport();
|
||||
if (staleBefore && !node.areBlockContentsStale()) {
|
||||
LOG.info("BLOCK* processReport: Received first block report from "
|
||||
+ node + " after becoming active. Its block contents are no longer"
|
||||
+ " considered stale");
|
||||
+ node + " after starting up or becoming active. Its block "
|
||||
+ "contents are no longer considered stale");
|
||||
rescanPostponedMisreplicatedBlocks();
|
||||
}
|
||||
|
||||
|
@ -3180,4 +3180,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|||
OK
|
||||
}
|
||||
|
||||
public void shutdown() {
|
||||
blocksMap.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -67,7 +67,7 @@ class BlocksMap {
|
|||
|
||||
|
||||
void close() {
|
||||
// Empty blocks once GSet#clear is implemented (HDFS-3940)
|
||||
blocks.clear();
|
||||
}
|
||||
|
||||
BlockCollection getBlockCollection(Block b) {
|
||||
|
|
|
@ -612,6 +612,7 @@ class BPOfferService {
|
|||
case DatanodeProtocol.DNA_TRANSFER:
|
||||
case DatanodeProtocol.DNA_INVALIDATE:
|
||||
case DatanodeProtocol.DNA_SHUTDOWN:
|
||||
case DatanodeProtocol.DNA_FINALIZE:
|
||||
case DatanodeProtocol.DNA_RECOVERBLOCK:
|
||||
case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
|
||||
LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
|
||||
|
|
|
@ -512,7 +512,7 @@ class BPServiceActor implements Runnable {
|
|||
//
|
||||
// Every so often, send heartbeat or block-report
|
||||
//
|
||||
if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
|
||||
if (startTime - lastHeartbeat >= dnConf.heartBeatInterval) {
|
||||
//
|
||||
// All heartbeat messages include following info:
|
||||
// -- Datanode name
|
||||
|
|
|
@ -160,8 +160,8 @@ class BlockPoolManager {
|
|||
Map<String, Map<String, InetSocketAddress>> addrMap) throws IOException {
|
||||
assert Thread.holdsLock(refreshNamenodesLock);
|
||||
|
||||
Set<String> toRefresh = Sets.newHashSet();
|
||||
Set<String> toAdd = Sets.newHashSet();
|
||||
Set<String> toRefresh = Sets.newLinkedHashSet();
|
||||
Set<String> toAdd = Sets.newLinkedHashSet();
|
||||
Set<String> toRemove;
|
||||
|
||||
synchronized (this) {
|
||||
|
|
|
@ -0,0 +1,259 @@
|
|||
/**
|
||||
* 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.fsdataset;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
||||
|
||||
/**
|
||||
* A DN volume choosing policy which takes into account the amount of free
|
||||
* space on each of the available volumes when considering where to assign a
|
||||
* new replica allocation. By default this policy prefers assigning replicas to
|
||||
* those volumes with more available free space, so as to over time balance the
|
||||
* available space of all the volumes within a DN.
|
||||
*/
|
||||
public class AvailableSpaceVolumeChoosingPolicy<V extends FsVolumeSpi>
|
||||
implements VolumeChoosingPolicy<V>, Configurable {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(AvailableSpaceVolumeChoosingPolicy.class);
|
||||
|
||||
private static final Random RAND = new Random();
|
||||
|
||||
private long balancedSpaceThreshold = DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_DEFAULT;
|
||||
private float balancedPreferencePercent = DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_DEFAULT;
|
||||
|
||||
@Override
|
||||
public synchronized void setConf(Configuration conf) {
|
||||
balancedSpaceThreshold = conf.getLong(
|
||||
DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY,
|
||||
DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_DEFAULT);
|
||||
balancedPreferencePercent = conf.getFloat(
|
||||
DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY,
|
||||
DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_DEFAULT);
|
||||
|
||||
LOG.info("Available space volume choosing policy initialized: " +
|
||||
DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY +
|
||||
" = " + balancedSpaceThreshold + ", " +
|
||||
DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY +
|
||||
" = " + balancedPreferencePercent);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Configuration getConf() {
|
||||
// Nothing to do. Only added to fulfill the Configurable contract.
|
||||
return null;
|
||||
}
|
||||
|
||||
private VolumeChoosingPolicy<V> roundRobinPolicyBalanced =
|
||||
new RoundRobinVolumeChoosingPolicy<V>();
|
||||
private VolumeChoosingPolicy<V> roundRobinPolicyHighAvailable =
|
||||
new RoundRobinVolumeChoosingPolicy<V>();
|
||||
private VolumeChoosingPolicy<V> roundRobinPolicyLowAvailable =
|
||||
new RoundRobinVolumeChoosingPolicy<V>();
|
||||
|
||||
@Override
|
||||
public synchronized V chooseVolume(List<V> volumes,
|
||||
final long replicaSize) throws IOException {
|
||||
if (volumes.size() < 1) {
|
||||
throw new DiskOutOfSpaceException("No more available volumes");
|
||||
}
|
||||
|
||||
AvailableSpaceVolumeList volumesWithSpaces =
|
||||
new AvailableSpaceVolumeList(volumes);
|
||||
|
||||
if (volumesWithSpaces.areAllVolumesWithinFreeSpaceThreshold()) {
|
||||
// If they're actually not too far out of whack, fall back on pure round
|
||||
// robin.
|
||||
V volume = roundRobinPolicyBalanced.chooseVolume(volumes, replicaSize);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("All volumes are within the configured free space balance " +
|
||||
"threshold. Selecting " + volume + " for write of block size " +
|
||||
replicaSize);
|
||||
}
|
||||
return volume;
|
||||
} else {
|
||||
V volume = null;
|
||||
// If none of the volumes with low free space have enough space for the
|
||||
// replica, always try to choose a volume with a lot of free space.
|
||||
long mostAvailableAmongLowVolumes = volumesWithSpaces
|
||||
.getMostAvailableSpaceAmongVolumesWithLowAvailableSpace();
|
||||
|
||||
List<V> highAvailableVolumes = extractVolumesFromPairs(
|
||||
volumesWithSpaces.getVolumesWithHighAvailableSpace());
|
||||
List<V> lowAvailableVolumes = extractVolumesFromPairs(
|
||||
volumesWithSpaces.getVolumesWithLowAvailableSpace());
|
||||
|
||||
float preferencePercentScaler =
|
||||
(highAvailableVolumes.size() * balancedPreferencePercent) +
|
||||
(lowAvailableVolumes.size() * (1 - balancedPreferencePercent));
|
||||
float scaledPreferencePercent =
|
||||
(highAvailableVolumes.size() * balancedPreferencePercent) /
|
||||
preferencePercentScaler;
|
||||
if (mostAvailableAmongLowVolumes < replicaSize ||
|
||||
RAND.nextFloat() < scaledPreferencePercent) {
|
||||
volume = roundRobinPolicyHighAvailable.chooseVolume(
|
||||
highAvailableVolumes,
|
||||
replicaSize);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Volumes are imbalanced. Selecting " + volume +
|
||||
" from high available space volumes for write of block size "
|
||||
+ replicaSize);
|
||||
}
|
||||
} else {
|
||||
volume = roundRobinPolicyLowAvailable.chooseVolume(
|
||||
lowAvailableVolumes,
|
||||
replicaSize);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Volumes are imbalanced. Selecting " + volume +
|
||||
" from low available space volumes for write of block size "
|
||||
+ replicaSize);
|
||||
}
|
||||
}
|
||||
return volume;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to keep track of the list of volumes we're choosing from.
|
||||
*/
|
||||
private class AvailableSpaceVolumeList {
|
||||
private final List<AvailableSpaceVolumePair> volumes;
|
||||
|
||||
public AvailableSpaceVolumeList(List<V> volumes) throws IOException {
|
||||
this.volumes = new ArrayList<AvailableSpaceVolumePair>();
|
||||
for (V volume : volumes) {
|
||||
this.volumes.add(new AvailableSpaceVolumePair(volume));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the available space on all the volumes is roughly equal.
|
||||
*
|
||||
* @param volumes the volumes to check
|
||||
* @return true if all volumes' free space is within the configured threshold,
|
||||
* false otherwise.
|
||||
* @throws IOException
|
||||
* in the event of error checking amount of available space
|
||||
*/
|
||||
public boolean areAllVolumesWithinFreeSpaceThreshold() {
|
||||
long leastAvailable = Long.MAX_VALUE;
|
||||
long mostAvailable = 0;
|
||||
for (AvailableSpaceVolumePair volume : volumes) {
|
||||
leastAvailable = Math.min(leastAvailable, volume.getAvailable());
|
||||
mostAvailable = Math.max(mostAvailable, volume.getAvailable());
|
||||
}
|
||||
return (mostAvailable - leastAvailable) < balancedSpaceThreshold;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the minimum amount of space available on a single volume,
|
||||
* across all volumes.
|
||||
*/
|
||||
private long getLeastAvailableSpace() {
|
||||
long leastAvailable = Long.MAX_VALUE;
|
||||
for (AvailableSpaceVolumePair volume : volumes) {
|
||||
leastAvailable = Math.min(leastAvailable, volume.getAvailable());
|
||||
}
|
||||
return leastAvailable;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the maximum amount of space available across volumes with low space.
|
||||
*/
|
||||
public long getMostAvailableSpaceAmongVolumesWithLowAvailableSpace() {
|
||||
long mostAvailable = Long.MIN_VALUE;
|
||||
for (AvailableSpaceVolumePair volume : getVolumesWithLowAvailableSpace()) {
|
||||
mostAvailable = Math.max(mostAvailable, volume.getAvailable());
|
||||
}
|
||||
return mostAvailable;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the list of volumes with relatively low available space.
|
||||
*/
|
||||
public List<AvailableSpaceVolumePair> getVolumesWithLowAvailableSpace() {
|
||||
long leastAvailable = getLeastAvailableSpace();
|
||||
List<AvailableSpaceVolumePair> ret = new ArrayList<AvailableSpaceVolumePair>();
|
||||
for (AvailableSpaceVolumePair volume : volumes) {
|
||||
if (volume.getAvailable() <= leastAvailable + balancedSpaceThreshold) {
|
||||
ret.add(volume);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the list of volumes with a lot of available space.
|
||||
*/
|
||||
public List<AvailableSpaceVolumePair> getVolumesWithHighAvailableSpace() {
|
||||
long leastAvailable = getLeastAvailableSpace();
|
||||
List<AvailableSpaceVolumePair> ret = new ArrayList<AvailableSpaceVolumePair>();
|
||||
for (AvailableSpaceVolumePair volume : volumes) {
|
||||
if (volume.getAvailable() > leastAvailable + balancedSpaceThreshold) {
|
||||
ret.add(volume);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Used so that we only check the available space on a given volume once, at
|
||||
* the beginning of {@link AvailableSpaceVolumeChoosingPolicy#chooseVolume(List, long)}.
|
||||
*/
|
||||
private class AvailableSpaceVolumePair {
|
||||
private final V volume;
|
||||
private final long availableSpace;
|
||||
|
||||
public AvailableSpaceVolumePair(V volume) throws IOException {
|
||||
this.volume = volume;
|
||||
this.availableSpace = volume.getAvailable();
|
||||
}
|
||||
|
||||
public long getAvailable() {
|
||||
return availableSpace;
|
||||
}
|
||||
|
||||
public V getVolume() {
|
||||
return volume;
|
||||
}
|
||||
}
|
||||
|
||||
private List<V> extractVolumesFromPairs(List<AvailableSpaceVolumePair> volumes) {
|
||||
List<V> ret = new ArrayList<V>();
|
||||
for (AvailableSpaceVolumePair volume : volumes) {
|
||||
ret.add(volume.getVolume());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
}
|
|
@ -2098,4 +2098,8 @@ public class FSDirectory implements Closeable {
|
|||
inode.setLocalName(name.getBytes());
|
||||
}
|
||||
}
|
||||
|
||||
void shutdown() {
|
||||
nameCache.reset();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -656,6 +656,7 @@ public class FSEditLog implements LogsPurgeable {
|
|||
*/
|
||||
public void logOpenFile(String path, INodeFileUnderConstruction newNode) {
|
||||
AddOp op = AddOp.getInstance(cache.get())
|
||||
.setInodeId(newNode.getId())
|
||||
.setPath(path)
|
||||
.setReplication(newNode.getBlockReplication())
|
||||
.setModificationTime(newNode.getModificationTime())
|
||||
|
@ -697,6 +698,7 @@ public class FSEditLog implements LogsPurgeable {
|
|||
*/
|
||||
public void logMkDir(String path, INode newNode) {
|
||||
MkdirOp op = MkdirOp.getInstance(cache.get())
|
||||
.setInodeId(newNode.getId())
|
||||
.setPath(path)
|
||||
.setTimestamp(newNode.getModificationTime())
|
||||
.setPermissionStatus(newNode.getPermissionStatus());
|
||||
|
@ -814,6 +816,7 @@ public class FSEditLog implements LogsPurgeable {
|
|||
void logSymlink(String path, String value, long mtime,
|
||||
long atime, INodeSymlink node) {
|
||||
SymlinkOp op = SymlinkOp.getInstance(cache.get())
|
||||
.setId(node.getId())
|
||||
.setPath(path)
|
||||
.setValue(value)
|
||||
.setModificationTime(mtime)
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage;
|
||||
|
@ -167,7 +168,7 @@ public class FSEditLogLoader {
|
|||
}
|
||||
}
|
||||
try {
|
||||
long inodeId = applyEditLogOp(op, fsDir, in.getVersion());
|
||||
long inodeId = applyEditLogOp(op, fsDir, in.getVersion(), lastInodeId);
|
||||
if (lastInodeId < inodeId) {
|
||||
lastInodeId = inodeId;
|
||||
}
|
||||
|
@ -223,9 +224,30 @@ public class FSEditLogLoader {
|
|||
return numEdits;
|
||||
}
|
||||
|
||||
// allocate and update last allocated inode id
|
||||
private long getAndUpdateLastInodeId(long inodeIdFromOp, int logVersion,
|
||||
long lastInodeId) throws IOException {
|
||||
long inodeId = inodeIdFromOp;
|
||||
|
||||
if (inodeId == INodeId.GRANDFATHER_INODE_ID) {
|
||||
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
|
||||
throw new IOException("The layout version " + logVersion
|
||||
+ " supports inodeId but gave bogus inodeId");
|
||||
}
|
||||
inodeId = fsNamesys.allocateNewInodeId();
|
||||
} else {
|
||||
// need to reset lastInodeId. fsnamesys gets lastInodeId firstly from
|
||||
// fsimage but editlog captures more recent inodeId allocations
|
||||
if (inodeId > lastInodeId) {
|
||||
fsNamesys.resetLastInodeId(inodeId);
|
||||
}
|
||||
}
|
||||
return inodeId;
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
|
||||
int logVersion) throws IOException {
|
||||
int logVersion, long lastInodeId) throws IOException {
|
||||
long inodeId = INodeId.GRANDFATHER_INODE_ID;
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("replaying edit log: " + op);
|
||||
|
@ -256,7 +278,8 @@ public class FSEditLogLoader {
|
|||
assert addCloseOp.blocks.length == 0;
|
||||
|
||||
// add to the file tree
|
||||
inodeId = fsNamesys.allocateNewInodeId();
|
||||
inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
|
||||
lastInodeId);
|
||||
newFile = (INodeFile) fsDir.unprotectedAddFile(inodeId,
|
||||
addCloseOp.path, addCloseOp.permissions, replication,
|
||||
addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
|
||||
|
@ -371,7 +394,8 @@ public class FSEditLogLoader {
|
|||
}
|
||||
case OP_MKDIR: {
|
||||
MkdirOp mkdirOp = (MkdirOp)op;
|
||||
inodeId = fsNamesys.allocateNewInodeId();
|
||||
inodeId = getAndUpdateLastInodeId(mkdirOp.inodeId, logVersion,
|
||||
lastInodeId);
|
||||
fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
|
||||
mkdirOp.timestamp);
|
||||
break;
|
||||
|
@ -425,7 +449,8 @@ public class FSEditLogLoader {
|
|||
}
|
||||
case OP_SYMLINK: {
|
||||
SymlinkOp symlinkOp = (SymlinkOp)op;
|
||||
inodeId = fsNamesys.allocateNewInodeId();
|
||||
inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
|
||||
lastInodeId);
|
||||
fsDir.unprotectedAddSymlink(inodeId, symlinkOp.path,
|
||||
symlinkOp.value, symlinkOp.mtime,
|
||||
symlinkOp.atime, symlinkOp.permissionStatus);
|
||||
|
|
|
@ -158,6 +158,7 @@ public abstract class FSEditLogOp {
|
|||
@SuppressWarnings("unchecked")
|
||||
static abstract class AddCloseOp extends FSEditLogOp implements BlockListUpdatingOp {
|
||||
int length;
|
||||
long inodeId;
|
||||
String path;
|
||||
short replication;
|
||||
long mtime;
|
||||
|
@ -173,6 +174,11 @@ public abstract class FSEditLogOp {
|
|||
assert(opCode == OP_ADD || opCode == OP_CLOSE);
|
||||
}
|
||||
|
||||
<T extends AddCloseOp> T setInodeId(long inodeId) {
|
||||
this.inodeId = inodeId;
|
||||
return (T)this;
|
||||
}
|
||||
|
||||
<T extends AddCloseOp> T setPath(String path) {
|
||||
this.path = path;
|
||||
return (T)this;
|
||||
|
@ -235,6 +241,7 @@ public abstract class FSEditLogOp {
|
|||
@Override
|
||||
public
|
||||
void writeFields(DataOutputStream out) throws IOException {
|
||||
FSImageSerialization.writeLong(inodeId, out);
|
||||
FSImageSerialization.writeString(path, out);
|
||||
FSImageSerialization.writeShort(replication, out);
|
||||
FSImageSerialization.writeLong(mtime, out);
|
||||
|
@ -255,6 +262,12 @@ public abstract class FSEditLogOp {
|
|||
if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
||||
this.length = in.readInt();
|
||||
}
|
||||
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
|
||||
this.inodeId = in.readLong();
|
||||
} else {
|
||||
// The inodeId should be updated when this editLogOp is applied
|
||||
this.inodeId = INodeId.GRANDFATHER_INODE_ID;
|
||||
}
|
||||
if ((-17 < logVersion && length != 4) ||
|
||||
(logVersion <= -17 && length != 5 && !LayoutVersion.supports(
|
||||
Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
|
||||
|
@ -327,6 +340,8 @@ public abstract class FSEditLogOp {
|
|||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("[length=");
|
||||
builder.append(length);
|
||||
builder.append(", inodeId=");
|
||||
builder.append(inodeId);
|
||||
builder.append(", path=");
|
||||
builder.append(path);
|
||||
builder.append(", replication=");
|
||||
|
@ -357,6 +372,8 @@ public abstract class FSEditLogOp {
|
|||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "LENGTH",
|
||||
Integer.valueOf(length).toString());
|
||||
XMLUtils.addSaxString(contentHandler, "INODEID",
|
||||
Long.valueOf(inodeId).toString());
|
||||
XMLUtils.addSaxString(contentHandler, "PATH", path);
|
||||
XMLUtils.addSaxString(contentHandler, "REPLICATION",
|
||||
Short.valueOf(replication).toString());
|
||||
|
@ -376,6 +393,7 @@ public abstract class FSEditLogOp {
|
|||
|
||||
@Override void fromXml(Stanza st) throws InvalidXmlException {
|
||||
this.length = Integer.valueOf(st.getValue("LENGTH"));
|
||||
this.inodeId = Long.valueOf(st.getValue("INODEID"));
|
||||
this.path = st.getValue("PATH");
|
||||
this.replication = Short.valueOf(st.getValue("REPLICATION"));
|
||||
this.mtime = Long.valueOf(st.getValue("MTIME"));
|
||||
|
@ -907,6 +925,7 @@ public abstract class FSEditLogOp {
|
|||
|
||||
static class MkdirOp extends FSEditLogOp {
|
||||
int length;
|
||||
long inodeId;
|
||||
String path;
|
||||
long timestamp;
|
||||
PermissionStatus permissions;
|
||||
|
@ -919,6 +938,11 @@ public abstract class FSEditLogOp {
|
|||
return (MkdirOp)cache.get(OP_MKDIR);
|
||||
}
|
||||
|
||||
MkdirOp setInodeId(long inodeId) {
|
||||
this.inodeId = inodeId;
|
||||
return this;
|
||||
}
|
||||
|
||||
MkdirOp setPath(String path) {
|
||||
this.path = path;
|
||||
return this;
|
||||
|
@ -937,6 +961,7 @@ public abstract class FSEditLogOp {
|
|||
@Override
|
||||
public
|
||||
void writeFields(DataOutputStream out) throws IOException {
|
||||
FSImageSerialization.writeLong(inodeId, out);
|
||||
FSImageSerialization.writeString(path, out);
|
||||
FSImageSerialization.writeLong(timestamp, out); // mtime
|
||||
FSImageSerialization.writeLong(timestamp, out); // atime, unused at this
|
||||
|
@ -953,6 +978,12 @@ public abstract class FSEditLogOp {
|
|||
&& !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
||||
throw new IOException("Incorrect data format. Mkdir operation.");
|
||||
}
|
||||
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
|
||||
this.inodeId = FSImageSerialization.readLong(in);
|
||||
} else {
|
||||
// This id should be updated when this editLogOp is applied
|
||||
this.inodeId = INodeId.GRANDFATHER_INODE_ID;
|
||||
}
|
||||
this.path = FSImageSerialization.readString(in);
|
||||
if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
|
||||
this.timestamp = FSImageSerialization.readLong(in);
|
||||
|
@ -979,6 +1010,8 @@ public abstract class FSEditLogOp {
|
|||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("MkdirOp [length=");
|
||||
builder.append(length);
|
||||
builder.append(", inodeId=");
|
||||
builder.append(inodeId);
|
||||
builder.append(", path=");
|
||||
builder.append(path);
|
||||
builder.append(", timestamp=");
|
||||
|
@ -997,6 +1030,8 @@ public abstract class FSEditLogOp {
|
|||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "LENGTH",
|
||||
Integer.valueOf(length).toString());
|
||||
XMLUtils.addSaxString(contentHandler, "INODEID",
|
||||
Long.valueOf(inodeId).toString());
|
||||
XMLUtils.addSaxString(contentHandler, "PATH", path);
|
||||
XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
|
||||
Long.valueOf(timestamp).toString());
|
||||
|
@ -1005,6 +1040,7 @@ public abstract class FSEditLogOp {
|
|||
|
||||
@Override void fromXml(Stanza st) throws InvalidXmlException {
|
||||
this.length = Integer.valueOf(st.getValue("LENGTH"));
|
||||
this.inodeId = Long.valueOf(st.getValue("INODEID"));
|
||||
this.path = st.getValue("PATH");
|
||||
this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
|
||||
this.permissions =
|
||||
|
@ -1483,6 +1519,7 @@ public abstract class FSEditLogOp {
|
|||
|
||||
static class SymlinkOp extends FSEditLogOp {
|
||||
int length;
|
||||
long inodeId;
|
||||
String path;
|
||||
String value;
|
||||
long mtime;
|
||||
|
@ -1497,6 +1534,11 @@ public abstract class FSEditLogOp {
|
|||
return (SymlinkOp)cache.get(OP_SYMLINK);
|
||||
}
|
||||
|
||||
SymlinkOp setId(long inodeId) {
|
||||
this.inodeId = inodeId;
|
||||
return this;
|
||||
}
|
||||
|
||||
SymlinkOp setPath(String path) {
|
||||
this.path = path;
|
||||
return this;
|
||||
|
@ -1525,6 +1567,7 @@ public abstract class FSEditLogOp {
|
|||
@Override
|
||||
public
|
||||
void writeFields(DataOutputStream out) throws IOException {
|
||||
FSImageSerialization.writeLong(inodeId, out);
|
||||
FSImageSerialization.writeString(path, out);
|
||||
FSImageSerialization.writeString(value, out);
|
||||
FSImageSerialization.writeLong(mtime, out);
|
||||
|
@ -1542,6 +1585,12 @@ public abstract class FSEditLogOp {
|
|||
+ "symlink operation.");
|
||||
}
|
||||
}
|
||||
if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
|
||||
this.inodeId = FSImageSerialization.readLong(in);
|
||||
} else {
|
||||
// This id should be updated when the editLogOp is applied
|
||||
this.inodeId = INodeId.GRANDFATHER_INODE_ID;
|
||||
}
|
||||
this.path = FSImageSerialization.readString(in);
|
||||
this.value = FSImageSerialization.readString(in);
|
||||
|
||||
|
@ -1560,6 +1609,8 @@ public abstract class FSEditLogOp {
|
|||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("SymlinkOp [length=");
|
||||
builder.append(length);
|
||||
builder.append(", inodeId=");
|
||||
builder.append(inodeId);
|
||||
builder.append(", path=");
|
||||
builder.append(path);
|
||||
builder.append(", value=");
|
||||
|
@ -1582,6 +1633,8 @@ public abstract class FSEditLogOp {
|
|||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "LENGTH",
|
||||
Integer.valueOf(length).toString());
|
||||
XMLUtils.addSaxString(contentHandler, "INODEID",
|
||||
Long.valueOf(inodeId).toString());
|
||||
XMLUtils.addSaxString(contentHandler, "PATH", path);
|
||||
XMLUtils.addSaxString(contentHandler, "VALUE", value);
|
||||
XMLUtils.addSaxString(contentHandler, "MTIME",
|
||||
|
@ -1593,6 +1646,7 @@ public abstract class FSEditLogOp {
|
|||
|
||||
@Override void fromXml(Stanza st) throws InvalidXmlException {
|
||||
this.length = Integer.valueOf(st.getValue("LENGTH"));
|
||||
this.inodeId = Long.valueOf(st.getValue("INODEID"));
|
||||
this.path = st.getValue("PATH");
|
||||
this.value = st.getValue("VALUE");
|
||||
this.mtime = Long.valueOf(st.getValue("MTIME"));
|
||||
|
|
|
@ -206,6 +206,20 @@ class FSImageFormat {
|
|||
imgTxId = 0;
|
||||
}
|
||||
|
||||
// read the last allocated inode id in the fsimage
|
||||
if (LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion)) {
|
||||
long lastInodeId = in.readLong();
|
||||
namesystem.resetLastInodeId(lastInodeId);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("load last allocated InodeId from fsimage:" + lastInodeId);
|
||||
}
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Old layout version doesn't have inode id."
|
||||
+ " Will assign new id for each inode.");
|
||||
}
|
||||
}
|
||||
|
||||
// read compression related info
|
||||
FSImageCompression compression;
|
||||
if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
|
||||
|
@ -216,8 +230,7 @@ class FSImageFormat {
|
|||
in = compression.unwrapInputStream(fin);
|
||||
|
||||
LOG.info("Loading image file " + curFile + " using " + compression);
|
||||
// reset INodeId. TODO: remove this after inodeId is persisted in fsimage
|
||||
namesystem.resetLastInodeIdWithoutChecking(INodeId.LAST_RESERVED_ID);
|
||||
|
||||
// load all inodes
|
||||
LOG.info("Number of files = " + numFiles);
|
||||
if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
|
||||
|
@ -385,7 +398,8 @@ class FSImageFormat {
|
|||
long blockSize = 0;
|
||||
|
||||
int imgVersion = getLayoutVersion();
|
||||
long inodeId = namesystem.allocateNewInodeId();
|
||||
long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ?
|
||||
in.readLong() : namesystem.allocateNewInodeId();
|
||||
|
||||
short replication = in.readShort();
|
||||
replication = namesystem.getBlockManager().adjustReplication(replication);
|
||||
|
@ -436,8 +450,8 @@ class FSImageFormat {
|
|||
LOG.info("Number of files under construction = " + size);
|
||||
|
||||
for (int i = 0; i < size; i++) {
|
||||
INodeFileUnderConstruction cons =
|
||||
FSImageSerialization.readINodeUnderConstruction(in);
|
||||
INodeFileUnderConstruction cons = FSImageSerialization
|
||||
.readINodeUnderConstruction(in, namesystem, getLayoutVersion());
|
||||
|
||||
// verify that file exists in namespace
|
||||
String path = cons.getLocalName();
|
||||
|
@ -566,6 +580,7 @@ class FSImageFormat {
|
|||
out.writeLong(fsDir.rootDir.numItemsInTree());
|
||||
out.writeLong(sourceNamesystem.getGenerationStamp());
|
||||
out.writeLong(context.getTxId());
|
||||
out.writeLong(sourceNamesystem.getLastInodeId());
|
||||
|
||||
// write compression info and set up compressed stream
|
||||
out = compression.writeHeaderAndWrapStream(fos);
|
||||
|
|
|
@ -29,6 +29,8 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
|
|||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.DeprecatedUTF8;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||
|
@ -79,8 +81,11 @@ public class FSImageSerialization {
|
|||
// from the input stream
|
||||
//
|
||||
static INodeFileUnderConstruction readINodeUnderConstruction(
|
||||
DataInputStream in) throws IOException {
|
||||
DataInputStream in, FSNamesystem fsNamesys, int imgVersion)
|
||||
throws IOException {
|
||||
byte[] name = readBytes(in);
|
||||
long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in
|
||||
.readLong() : fsNamesys.allocateNewInodeId();
|
||||
short blockReplication = in.readShort();
|
||||
long modificationTime = in.readLong();
|
||||
long preferredBlockSize = in.readLong();
|
||||
|
@ -107,8 +112,7 @@ public class FSImageSerialization {
|
|||
int numLocs = in.readInt();
|
||||
assert numLocs == 0 : "Unexpected block locations";
|
||||
|
||||
//TODO: get inodeId from fsimage after inodeId is persisted
|
||||
return new INodeFileUnderConstruction(INodeId.GRANDFATHER_INODE_ID,
|
||||
return new INodeFileUnderConstruction(inodeId,
|
||||
name,
|
||||
blockReplication,
|
||||
modificationTime,
|
||||
|
@ -128,6 +132,7 @@ public class FSImageSerialization {
|
|||
String path)
|
||||
throws IOException {
|
||||
writeString(path, out);
|
||||
out.writeLong(cons.getId());
|
||||
out.writeShort(cons.getBlockReplication());
|
||||
out.writeLong(cons.getModificationTime());
|
||||
out.writeLong(cons.getPreferredBlockSize());
|
||||
|
@ -151,6 +156,7 @@ public class FSImageSerialization {
|
|||
byte[] name = node.getLocalNameBytes();
|
||||
out.writeShort(name.length);
|
||||
out.write(name);
|
||||
out.writeLong(node.getId());
|
||||
FsPermission filePerm = TL_DATA.get().FILE_PERM;
|
||||
if (node.isDirectory()) {
|
||||
out.writeShort(0); // replication
|
||||
|
|
|
@ -1210,7 +1210,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot set permission for " + src, safeMode);
|
||||
}
|
||||
|
@ -1248,7 +1247,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot set owner for " + src, safeMode);
|
||||
}
|
||||
|
@ -1302,9 +1300,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
LocatedBlocks getBlockLocations(String src, long offset, long length,
|
||||
boolean doAccessTime, boolean needBlockToken, boolean checkSafeMode)
|
||||
throws FileNotFoundException, UnresolvedLinkException, IOException {
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
try {
|
||||
return getBlockLocationsInt(pc, src, offset, length, doAccessTime,
|
||||
return getBlockLocationsInt(src, offset, length, doAccessTime,
|
||||
needBlockToken, checkSafeMode);
|
||||
} catch (AccessControlException e) {
|
||||
logAuditEvent(false, "open", src);
|
||||
|
@ -1312,14 +1309,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
}
|
||||
}
|
||||
|
||||
private LocatedBlocks getBlockLocationsInt(FSPermissionChecker pc,
|
||||
String src, long offset, long length, boolean doAccessTime,
|
||||
boolean needBlockToken, boolean checkSafeMode)
|
||||
private LocatedBlocks getBlockLocationsInt(String src, long offset,
|
||||
long length, boolean doAccessTime, boolean needBlockToken,
|
||||
boolean checkSafeMode)
|
||||
throws FileNotFoundException, UnresolvedLinkException, IOException {
|
||||
if (isPermissionEnabled) {
|
||||
checkPathAccess(pc, src, FsAction.READ);
|
||||
}
|
||||
|
||||
if (offset < 0) {
|
||||
throw new HadoopIllegalArgumentException(
|
||||
"Negative offset is not supported. File: " + src);
|
||||
|
@ -1347,13 +1340,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
* Get block locations within the specified range, updating the
|
||||
* access times if necessary.
|
||||
*/
|
||||
private LocatedBlocks getBlockLocationsUpdateTimes(String src,
|
||||
long offset,
|
||||
long length,
|
||||
boolean doAccessTime,
|
||||
boolean needBlockToken)
|
||||
throws FileNotFoundException, UnresolvedLinkException, IOException {
|
||||
|
||||
private LocatedBlocks getBlockLocationsUpdateTimes(String src, long offset,
|
||||
long length, boolean doAccessTime, boolean needBlockToken)
|
||||
throws FileNotFoundException,
|
||||
UnresolvedLinkException, IOException {
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
for (int attempt = 0; attempt < 2; attempt++) {
|
||||
boolean isReadOp = (attempt == 0);
|
||||
if (isReadOp) { // first attempt is with readlock
|
||||
|
@ -1369,6 +1360,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
} else {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
}
|
||||
if (isPermissionEnabled) {
|
||||
checkPathAccess(pc, src, FsAction.READ);
|
||||
}
|
||||
|
||||
// if the namenode is in safemode, then do not update access time
|
||||
if (isInSafeMode()) {
|
||||
|
@ -1378,15 +1372,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
long now = now();
|
||||
final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
|
||||
if (doAccessTime && isAccessTimeSupported()) {
|
||||
if (now <= inode.getAccessTime() + getAccessTimePrecision()) {
|
||||
if (now > inode.getAccessTime() + getAccessTimePrecision()) {
|
||||
// if we have to set access time but we only have the readlock, then
|
||||
// restart this entire operation with the writeLock.
|
||||
if (isReadOp) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
dir.setTimes(src, inode, -1, now, false);
|
||||
}
|
||||
}
|
||||
return blockManager.createLocatedBlocks(inode.getBlocks(),
|
||||
inode.computeFileSize(false), inode.isUnderConstruction(),
|
||||
offset, length, needBlockToken);
|
||||
|
@ -1411,6 +1405,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
*/
|
||||
void concat(String target, String [] srcs)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
if(FSNamesystem.LOG.isDebugEnabled()) {
|
||||
FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) +
|
||||
" to " + target);
|
||||
}
|
||||
try {
|
||||
concatInt(target, srcs);
|
||||
} catch (AccessControlException e) {
|
||||
|
@ -1421,11 +1419,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
|
||||
private void concatInt(String target, String [] srcs)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
if(FSNamesystem.LOG.isDebugEnabled()) {
|
||||
FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) +
|
||||
" to " + target);
|
||||
}
|
||||
|
||||
// verify args
|
||||
if(target.isEmpty()) {
|
||||
throw new IllegalArgumentException("Target file name is empty");
|
||||
|
@ -1574,6 +1567,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
*/
|
||||
void setTimes(String src, long mtime, long atime)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
if (!isAccessTimeSupported() && atime != -1) {
|
||||
throw new IOException("Access time for hdfs is not configured. " +
|
||||
" Please set " + DFS_NAMENODE_ACCESSTIME_PRECISION_KEY + " configuration parameter.");
|
||||
}
|
||||
try {
|
||||
setTimesInt(src, mtime, atime);
|
||||
} catch (AccessControlException e) {
|
||||
|
@ -1584,16 +1581,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
|
||||
private void setTimesInt(String src, long mtime, long atime)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
if (!isAccessTimeSupported() && atime != -1) {
|
||||
throw new IOException("Access time for hdfs is not configured. " +
|
||||
" Please set " + DFS_NAMENODE_ACCESSTIME_PRECISION_KEY + " configuration parameter.");
|
||||
}
|
||||
HdfsFileStatus resultingStat = null;
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot set times " + src, safeMode);
|
||||
}
|
||||
|
||||
// Write access is required to set access and modification times
|
||||
if (isPermissionEnabled) {
|
||||
|
@ -1618,6 +1614,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
void createSymlink(String target, String link,
|
||||
PermissionStatus dirPerms, boolean createParent)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
if (!DFSUtil.isValidName(link)) {
|
||||
throw new InvalidPathException("Invalid file name: " + link);
|
||||
}
|
||||
try {
|
||||
createSymlinkInt(target, link, dirPerms, createParent);
|
||||
} catch (AccessControlException e) {
|
||||
|
@ -1629,41 +1628,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
private void createSymlinkInt(String target, String link,
|
||||
PermissionStatus dirPerms, boolean createParent)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.createSymlink: target="
|
||||
+ target + " link=" + link);
|
||||
}
|
||||
HdfsFileStatus resultingStat = null;
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
if (!createParent) {
|
||||
verifyParentDir(link);
|
||||
}
|
||||
createSymlinkInternal(pc, target, link, dirPerms, createParent);
|
||||
resultingStat = getAuditFileInfo(link, false);
|
||||
} finally {
|
||||
writeUnlock();
|
||||
}
|
||||
getEditLog().logSync();
|
||||
logAuditEvent(true, "createSymlink", link, target, resultingStat);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a symbolic link.
|
||||
*/
|
||||
private void createSymlinkInternal(FSPermissionChecker pc, String target,
|
||||
String link, PermissionStatus dirPerms, boolean createParent)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
assert hasWriteLock();
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.createSymlink: target=" +
|
||||
target + " link=" + link);
|
||||
}
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot create symlink " + link, safeMode);
|
||||
}
|
||||
if (!DFSUtil.isValidName(link)) {
|
||||
throw new InvalidPathException("Invalid file name: " + link);
|
||||
if (!createParent) {
|
||||
verifyParentDir(link);
|
||||
}
|
||||
if (!dir.isValidToCreate(link)) {
|
||||
throw new IOException("failed to create link " + link
|
||||
|
@ -1677,6 +1656,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
|
||||
// add symbolic link to namespace
|
||||
dir.addSymlink(link, target, dirPerms, createParent);
|
||||
resultingStat = getAuditFileInfo(link, false);
|
||||
} finally {
|
||||
writeUnlock();
|
||||
}
|
||||
getEditLog().logSync();
|
||||
logAuditEvent(true, "createSymlink", link, target, resultingStat);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1798,13 +1783,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
throws AccessControlException, SafeModeException,
|
||||
FileAlreadyExistsException, UnresolvedLinkException,
|
||||
FileNotFoundException, ParentNotDirectoryException, IOException {
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: src=" + src
|
||||
+ ", holder=" + holder
|
||||
+ ", clientMachine=" + clientMachine
|
||||
+ ", createParent=" + createParent
|
||||
+ ", replication=" + replication
|
||||
+ ", createFlag=" + flag.toString());
|
||||
}
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new InvalidPathException(src);
|
||||
}
|
||||
|
||||
boolean skipSync = false;
|
||||
final HdfsFileStatus stat;
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
startFileInternal(pc, src, permissions, holder, clientMachine, flag,
|
||||
createParent, replication, blockSize);
|
||||
stat = dir.getFileInfo(src, false);
|
||||
|
@ -1847,21 +1843,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
AccessControlException, UnresolvedLinkException, FileNotFoundException,
|
||||
ParentNotDirectoryException, IOException {
|
||||
assert hasWriteLock();
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: src=" + src
|
||||
+ ", holder=" + holder
|
||||
+ ", clientMachine=" + clientMachine
|
||||
+ ", createParent=" + createParent
|
||||
+ ", replication=" + replication
|
||||
+ ", createFlag=" + flag.toString());
|
||||
}
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot create file" + src, safeMode);
|
||||
}
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new InvalidPathException(src);
|
||||
}
|
||||
|
||||
// Verify that the destination does not exist as a directory already.
|
||||
boolean pathExists = dir.exists(src);
|
||||
if (pathExists && dir.isDir(src)) {
|
||||
|
@ -1997,21 +1982,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
*/
|
||||
boolean recoverLease(String src, String holder, String clientMachine)
|
||||
throws IOException {
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new IOException("Invalid file name: " + src);
|
||||
}
|
||||
|
||||
boolean skipSync = false;
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException(
|
||||
"Cannot recover the lease of " + src, safeMode);
|
||||
}
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new IOException("Invalid file name: " + src);
|
||||
}
|
||||
|
||||
final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
|
||||
if (!inode.isUnderConstruction()) {
|
||||
return true;
|
||||
|
@ -2135,13 +2119,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
"Append is not enabled on this NameNode. Use the " +
|
||||
DFS_SUPPORT_APPEND_KEY + " configuration option to enable it.");
|
||||
}
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src=" + src
|
||||
+ ", holder=" + holder
|
||||
+ ", clientMachine=" + clientMachine);
|
||||
}
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new InvalidPathException(src);
|
||||
}
|
||||
|
||||
LocatedBlock lb = null;
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
lb = startFileInternal(pc, src, null, holder, clientMachine,
|
||||
EnumSet.of(CreateFlag.APPEND),
|
||||
false, blockManager.maxReplication, 0);
|
||||
|
@ -2434,21 +2425,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
boolean abandonBlock(ExtendedBlock b, String src, String holder)
|
||||
throws LeaseExpiredException, FileNotFoundException,
|
||||
UnresolvedLinkException, IOException {
|
||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: " + b
|
||||
+ "of file " + src);
|
||||
}
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
//
|
||||
// Remove the block from the pending creates list
|
||||
//
|
||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
|
||||
+b+"of file "+src);
|
||||
}
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot abandon block " + b +
|
||||
" for fle" + src, safeMode);
|
||||
}
|
||||
//
|
||||
// Remove the block from the pending creates list
|
||||
//
|
||||
INodeFileUnderConstruction file = checkLease(src, holder);
|
||||
dir.removeBlock(src, file, ExtendedBlock.getLocalBlock(b));
|
||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
|
@ -2510,19 +2501,23 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
*/
|
||||
boolean completeFile(String src, String holder, ExtendedBlock last)
|
||||
throws SafeModeException, UnresolvedLinkException, IOException {
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " +
|
||||
src + " for " + holder);
|
||||
}
|
||||
checkBlock(last);
|
||||
boolean success = false;
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
success = completeFileInternal(src, holder,
|
||||
ExtendedBlock.getLocalBlock(last));
|
||||
} finally {
|
||||
writeUnlock();
|
||||
}
|
||||
getEditLog().logSync();
|
||||
NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
|
||||
+ holder);
|
||||
return success;
|
||||
}
|
||||
|
||||
|
@ -2530,10 +2525,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
String holder, Block last) throws SafeModeException,
|
||||
UnresolvedLinkException, IOException {
|
||||
assert hasWriteLock();
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " +
|
||||
src + " for " + holder);
|
||||
}
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot complete file " + src, safeMode);
|
||||
}
|
||||
|
@ -2569,9 +2561,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
}
|
||||
|
||||
finalizeINodeFileUnderConstruction(src, pendingFile);
|
||||
|
||||
NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
|
||||
+ holder);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -2672,18 +2661,19 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
|
||||
private boolean renameToInt(String src, String dst)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
boolean status = false;
|
||||
HdfsFileStatus resultingStat = null;
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src +
|
||||
" to " + dst);
|
||||
}
|
||||
if (!DFSUtil.isValidName(dst)) {
|
||||
throw new IOException("Invalid name: " + dst);
|
||||
}
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
boolean status = false;
|
||||
HdfsFileStatus resultingStat = null;
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
status = renameToInternal(pc, src, dst);
|
||||
if (status) {
|
||||
resultingStat = getAuditFileInfo(dst, false);
|
||||
|
@ -2703,12 +2693,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
private boolean renameToInternal(FSPermissionChecker pc, String src, String dst)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
assert hasWriteLock();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot rename " + src, safeMode);
|
||||
}
|
||||
if (!DFSUtil.isValidName(dst)) {
|
||||
throw new IOException("Invalid name: " + dst);
|
||||
}
|
||||
if (isPermissionEnabled) {
|
||||
//We should not be doing this. This is move() not renameTo().
|
||||
//but for now,
|
||||
|
@ -2730,16 +2718,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
/** Rename src to dst */
|
||||
void renameTo(String src, String dst, Options.Rename... options)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
HdfsFileStatus resultingStat = null;
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
|
||||
+ src + " to " + dst);
|
||||
}
|
||||
if (!DFSUtil.isValidName(dst)) {
|
||||
throw new InvalidPathException("Invalid name: " + dst);
|
||||
}
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
HdfsFileStatus resultingStat = null;
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
renameToInternal(pc, src, dst, options);
|
||||
resultingStat = getAuditFileInfo(dst, false);
|
||||
} finally {
|
||||
|
@ -2758,12 +2748,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
private void renameToInternal(FSPermissionChecker pc, String src, String dst,
|
||||
Options.Rename... options) throws IOException {
|
||||
assert hasWriteLock();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot rename " + src, safeMode);
|
||||
}
|
||||
if (!DFSUtil.isValidName(dst)) {
|
||||
throw new InvalidPathException("Invalid name: " + dst);
|
||||
}
|
||||
if (isPermissionEnabled) {
|
||||
checkParentAccess(pc, src, FsAction.WRITE);
|
||||
checkAncestorAccess(pc, dst, FsAction.WRITE);
|
||||
|
@ -2950,16 +2938,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
HdfsFileStatus getFileInfo(String src, boolean resolveLink)
|
||||
throws AccessControlException, UnresolvedLinkException,
|
||||
StandbyException, IOException {
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new InvalidPathException("Invalid file name: " + src);
|
||||
}
|
||||
HdfsFileStatus stat = null;
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.READ);
|
||||
readLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.READ);
|
||||
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new InvalidPathException("Invalid file name: " + src);
|
||||
}
|
||||
if (isPermissionEnabled) {
|
||||
checkTraverse(pc, src);
|
||||
}
|
||||
|
@ -2974,6 +2961,33 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
return stat;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the file is closed
|
||||
*/
|
||||
boolean isFileClosed(String src)
|
||||
throws AccessControlException, UnresolvedLinkException,
|
||||
StandbyException, IOException {
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.READ);
|
||||
readLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.READ);
|
||||
if (isPermissionEnabled) {
|
||||
checkTraverse(pc, src);
|
||||
}
|
||||
return !INodeFile.valueOf(dir.getINode(src), src).isUnderConstruction();
|
||||
} catch (AccessControlException e) {
|
||||
if (isAuditEnabled() && isExternalInvocation()) {
|
||||
logAuditEvent(false, UserGroupInformation.getCurrentUser(),
|
||||
getRemoteIp(),
|
||||
"isFileClosed", src, null, null);
|
||||
}
|
||||
throw e;
|
||||
} finally {
|
||||
readUnlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create all the necessary directories
|
||||
*/
|
||||
|
@ -2989,16 +3003,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
|
||||
private boolean mkdirsInt(String src, PermissionStatus permissions,
|
||||
boolean createParent) throws IOException, UnresolvedLinkException {
|
||||
HdfsFileStatus resultingStat = null;
|
||||
boolean status = false;
|
||||
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
|
||||
}
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new InvalidPathException(src);
|
||||
}
|
||||
FSPermissionChecker pc = getPermissionChecker();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
HdfsFileStatus resultingStat = null;
|
||||
boolean status = false;
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
status = mkdirsInternal(pc, src, permissions, createParent);
|
||||
if (status) {
|
||||
resultingStat = dir.getFileInfo(src, false);
|
||||
|
@ -3020,6 +3036,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
PermissionStatus permissions, boolean createParent)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
assert hasWriteLock();
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot create directory " + src, safeMode);
|
||||
}
|
||||
|
@ -3031,9 +3048,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
// a new directory is not created.
|
||||
return true;
|
||||
}
|
||||
if (!DFSUtil.isValidName(src)) {
|
||||
throw new InvalidPathException(src);
|
||||
}
|
||||
if (isPermissionEnabled) {
|
||||
checkAncestorAccess(pc, src, FsAction.WRITE);
|
||||
}
|
||||
|
@ -3304,8 +3318,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
boolean closeFile, boolean deleteblock, DatanodeID[] newtargets,
|
||||
String[] newtargetstorages)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
String src = "";
|
||||
LOG.info("commitBlockSynchronization(lastblock=" + lastblock
|
||||
+ ", newgenerationstamp=" + newgenerationstamp
|
||||
+ ", newlength=" + newlength
|
||||
+ ", newtargets=" + Arrays.asList(newtargets)
|
||||
+ ", closeFile=" + closeFile
|
||||
+ ", deleteBlock=" + deleteblock
|
||||
+ ")");
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
String src = "";
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
@ -3317,13 +3338,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
"Cannot commitBlockSynchronization while in safe mode",
|
||||
safeMode);
|
||||
}
|
||||
LOG.info("commitBlockSynchronization(lastblock=" + lastblock
|
||||
+ ", newgenerationstamp=" + newgenerationstamp
|
||||
+ ", newlength=" + newlength
|
||||
+ ", newtargets=" + Arrays.asList(newtargets)
|
||||
+ ", closeFile=" + closeFile
|
||||
+ ", deleteBlock=" + deleteblock
|
||||
+ ")");
|
||||
final BlockInfo storedBlock = blockManager.getStoredBlock(ExtendedBlock
|
||||
.getLocalBlock(lastblock));
|
||||
if (storedBlock == null) {
|
||||
|
@ -3413,7 +3427,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot renew lease for " + holder, safeMode);
|
||||
}
|
||||
|
@ -4793,9 +4806,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
* shutdown FSNamesystem
|
||||
*/
|
||||
void shutdown() {
|
||||
if (mbeanName != null)
|
||||
if (mbeanName != null) {
|
||||
MBeans.unregister(mbeanName);
|
||||
}
|
||||
if (dir != null) {
|
||||
dir.shutdown();
|
||||
}
|
||||
if (blockManager != null) {
|
||||
blockManager.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override // FSNamesystemMBean
|
||||
|
@ -4882,11 +4902,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
*/
|
||||
void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
NameNode.stateChangeLog.info("*DIR* reportBadBlocks");
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
NameNode.stateChangeLog.info("*DIR* reportBadBlocks");
|
||||
for (int i = 0; i < blocks.length; i++) {
|
||||
ExtendedBlock blk = blocks[i].getBlock();
|
||||
DatanodeInfo[] nodes = blocks[i].getLocations();
|
||||
|
@ -4949,6 +4968,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
ExtendedBlock newBlock, DatanodeID[] newNodes)
|
||||
throws IOException {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
LOG.info("updatePipeline(block=" + oldBlock
|
||||
+ ", newGenerationStamp=" + newBlock.getGenerationStamp()
|
||||
+ ", newLength=" + newBlock.getNumBytes()
|
||||
+ ", newNodes=" + Arrays.asList(newNodes)
|
||||
+ ", clientName=" + clientName
|
||||
+ ")");
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
@ -4958,12 +4983,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
}
|
||||
assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
|
||||
+ oldBlock + " has different block identifier";
|
||||
LOG.info("updatePipeline(block=" + oldBlock
|
||||
+ ", newGenerationStamp=" + newBlock.getGenerationStamp()
|
||||
+ ", newLength=" + newBlock.getNumBytes()
|
||||
+ ", newNodes=" + Arrays.asList(newNodes)
|
||||
+ ", clientName=" + clientName
|
||||
+ ")");
|
||||
updatePipelineInternal(clientName, oldBlock, newBlock, newNodes);
|
||||
} finally {
|
||||
writeUnlock();
|
||||
|
@ -5215,7 +5234,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException("Cannot issue delegation token", safeMode);
|
||||
}
|
||||
|
@ -5340,6 +5358,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
getEditLog().logSync();
|
||||
}
|
||||
|
||||
/**
|
||||
* Log the cancellation of expired tokens to edit logs
|
||||
*
|
||||
* @param id token identifier to cancel
|
||||
*/
|
||||
public void logExpireDelegationToken(DelegationTokenIdentifier id) {
|
||||
assert !isInSafeMode() :
|
||||
"this should never be called while in safemode, since we stop " +
|
||||
"the DT manager before entering safemode!";
|
||||
// No need to hold FSN lock since we don't access any internal
|
||||
// structures, and this is stopped before the FSN shuts itself
|
||||
// down, etc.
|
||||
getEditLog().logCancelDelegationToken(id);
|
||||
}
|
||||
|
||||
private void logReassignLease(String leaseHolder, String src,
|
||||
String newHolder) {
|
||||
assert hasWriteLock();
|
||||
|
|
|
@ -691,6 +691,11 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
|||
return namesystem.getFileInfo(src, true);
|
||||
}
|
||||
|
||||
@Override // ClientProtocol
|
||||
public boolean isFileClosed(String src) throws IOException{
|
||||
return namesystem.isFileClosed(src);
|
||||
}
|
||||
|
||||
@Override // ClientProtocol
|
||||
public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
|
||||
metrics.incrFileInfoOps();
|
||||
|
|
|
@ -718,9 +718,15 @@ public class NamenodeWebHdfsMethods {
|
|||
|
||||
private static StreamingOutput getListingStream(final NamenodeProtocols np,
|
||||
final String p) throws IOException {
|
||||
final DirectoryListing first = getDirectoryListing(np, p,
|
||||
// allows exceptions like FNF or ACE to prevent http response of 200 for
|
||||
// a failure since we can't (currently) return error responses in the
|
||||
// middle of a streaming operation
|
||||
final DirectoryListing firstDirList = getDirectoryListing(np, p,
|
||||
HdfsFileStatus.EMPTY_NAME);
|
||||
|
||||
// must save ugi because the streaming object will be executed outside
|
||||
// the remote user's ugi
|
||||
final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
||||
return new StreamingOutput() {
|
||||
@Override
|
||||
public void write(final OutputStream outstream) throws IOException {
|
||||
|
@ -729,21 +735,32 @@ public class NamenodeWebHdfsMethods {
|
|||
out.println("{\"" + FileStatus.class.getSimpleName() + "es\":{\""
|
||||
+ FileStatus.class.getSimpleName() + "\":[");
|
||||
|
||||
final HdfsFileStatus[] partial = first.getPartialListing();
|
||||
if (partial.length > 0) {
|
||||
out.print(JsonUtil.toJsonString(partial[0], false));
|
||||
}
|
||||
for(int i = 1; i < partial.length; i++) {
|
||||
try {
|
||||
// restore remote user's ugi
|
||||
ugi.doAs(new PrivilegedExceptionAction<Void>() {
|
||||
@Override
|
||||
public Void run() throws IOException {
|
||||
long n = 0;
|
||||
for (DirectoryListing dirList = firstDirList; ;
|
||||
dirList = getDirectoryListing(np, p, dirList.getLastName())
|
||||
) {
|
||||
// send each segment of the directory listing
|
||||
for (HdfsFileStatus s : dirList.getPartialListing()) {
|
||||
if (n++ > 0) {
|
||||
out.println(',');
|
||||
out.print(JsonUtil.toJsonString(partial[i], false));
|
||||
}
|
||||
|
||||
for(DirectoryListing curr = first; curr.hasMore(); ) {
|
||||
curr = getDirectoryListing(np, p, curr.getLastName());
|
||||
for(HdfsFileStatus s : curr.getPartialListing()) {
|
||||
out.println(',');
|
||||
out.print(JsonUtil.toJsonString(s, false));
|
||||
}
|
||||
// stop if last segment
|
||||
if (!dirList.hasMore()) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
});
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
out.println();
|
||||
|
|
|
@ -123,7 +123,7 @@ class ImageLoaderCurrent implements ImageLoader {
|
|||
new SimpleDateFormat("yyyy-MM-dd HH:mm");
|
||||
private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
|
||||
-24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
|
||||
-40};
|
||||
-40, -41, -42};
|
||||
private int imageVersion = 0;
|
||||
|
||||
/* (non-Javadoc)
|
||||
|
@ -163,6 +163,10 @@ class ImageLoaderCurrent implements ImageLoader {
|
|||
v.visit(ImageElement.TRANSACTION_ID, in.readLong());
|
||||
}
|
||||
|
||||
if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
|
||||
v.visit(ImageElement.LAST_INODE_ID, in.readLong());
|
||||
}
|
||||
|
||||
if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
|
||||
boolean isCompressed = in.readBoolean();
|
||||
v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
|
||||
|
@ -440,6 +444,9 @@ class ImageLoaderCurrent implements ImageLoader {
|
|||
}
|
||||
|
||||
v.visit(ImageElement.INODE_PATH, pathName);
|
||||
if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
|
||||
v.visit(ImageElement.INODE_ID, in.readLong());
|
||||
}
|
||||
v.visit(ImageElement.REPLICATION, in.readShort());
|
||||
v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
|
||||
if(LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion))
|
||||
|
|
|
@ -80,7 +80,9 @@ abstract class ImageVisitor {
|
|||
DELEGATION_TOKEN_IDENTIFIER_MAX_DATE,
|
||||
DELEGATION_TOKEN_IDENTIFIER_EXPIRY_TIME,
|
||||
DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
|
||||
TRANSACTION_ID
|
||||
TRANSACTION_ID,
|
||||
LAST_INODE_ID,
|
||||
INODE_ID
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -81,4 +81,6 @@ public interface GSet<K, E extends K> extends Iterable<E> {
|
|||
* @throws NullPointerException if key == null.
|
||||
*/
|
||||
E remove(K key);
|
||||
|
||||
void clear();
|
||||
}
|
||||
|
|
|
@ -65,4 +65,9 @@ public class GSetByHashMap<K, E extends K> implements GSet<K, E> {
|
|||
public Iterator<E> iterator() {
|
||||
return m.values().iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
m.clear();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -85,7 +85,6 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
|
|||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("recommended=" + recommended_length + ", actual=" + actual);
|
||||
}
|
||||
|
||||
entries = new LinkedElement[actual];
|
||||
hash_mask = entries.length - 1;
|
||||
}
|
||||
|
@ -329,13 +328,18 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
|
|||
final int exponent = e2 < 0? 0: e2 > 30? 30: e2;
|
||||
final int c = 1 << exponent;
|
||||
|
||||
if (LightWeightGSet.LOG.isDebugEnabled()) {
|
||||
LOG.debug("Computing capacity for map " + mapName);
|
||||
LOG.debug("VM type = " + vmBit + "-bit");
|
||||
LOG.debug(percentage + "% max memory = "
|
||||
LOG.info("Computing capacity for map " + mapName);
|
||||
LOG.info("VM type = " + vmBit + "-bit");
|
||||
LOG.info(percentage + "% max memory = "
|
||||
+ StringUtils.TraditionalBinaryPrefix.long2String(maxMemory, "B", 1));
|
||||
LOG.debug("capacity = 2^" + exponent + " = " + c + " entries");
|
||||
}
|
||||
LOG.info("capacity = 2^" + exponent + " = " + c + " entries");
|
||||
return c;
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
for (int i = 0; i < entries.length; i++) {
|
||||
entries[i] = null;
|
||||
}
|
||||
size = 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -156,7 +156,6 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
private URI uri;
|
||||
private boolean hasInitedToken;
|
||||
private Token<?> delegationToken;
|
||||
private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
|
||||
private RetryPolicy retryPolicy = null;
|
||||
private Path workingDir;
|
||||
|
||||
|
@ -481,6 +480,8 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
try {
|
||||
if (op.getRequireAuth()) {
|
||||
LOG.debug("open AuthenticatedURL connection");
|
||||
UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
|
||||
final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
|
||||
conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
|
||||
} else {
|
||||
LOG.debug("open URL connection");
|
||||
|
@ -1006,20 +1007,12 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
@Override
|
||||
public long renew(final Token<?> token, final Configuration conf
|
||||
) throws IOException, InterruptedException {
|
||||
final UserGroupInformation ugi = UserGroupInformation.getLoginUser();
|
||||
// update the kerberos credentials, if they are coming from a keytab
|
||||
ugi.reloginFromKeytab();
|
||||
|
||||
return getWebHdfs(token, conf).renewDelegationToken(token);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancel(final Token<?> token, final Configuration conf
|
||||
) throws IOException, InterruptedException {
|
||||
final UserGroupInformation ugi = UserGroupInformation.getLoginUser();
|
||||
// update the kerberos credentials, if they are coming from a keytab
|
||||
ugi.checkTGTAndReloginFromKeytab();
|
||||
|
||||
getWebHdfs(token, conf).cancelDelegationToken(token);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -332,6 +332,14 @@ message GetFileInfoResponseProto {
|
|||
optional HdfsFileStatusProto fs = 1;
|
||||
}
|
||||
|
||||
message IsFileClosedRequestProto {
|
||||
required string src = 1;
|
||||
}
|
||||
|
||||
message IsFileClosedResponseProto {
|
||||
required bool result = 1;
|
||||
}
|
||||
|
||||
message GetFileLinkInfoRequestProto {
|
||||
required string src = 1;
|
||||
}
|
||||
|
@ -498,4 +506,6 @@ service ClientNamenodeProtocol {
|
|||
returns(SetBalancerBandwidthResponseProto);
|
||||
rpc getDataEncryptionKey(GetDataEncryptionKeyRequestProto)
|
||||
returns(GetDataEncryptionKeyResponseProto);
|
||||
rpc isFileClosed(IsFileClosedRequestProto)
|
||||
returns(IsFileClosedResponseProto);
|
||||
}
|
||||
|
|
|
@ -1242,4 +1242,32 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.datanode.fsdataset.volume.choosing.balanced-space-threshold</name>
|
||||
<value>10737418240</value> <!-- 10 GB -->
|
||||
<description>
|
||||
Only used when the dfs.datanode.fsdataset.volume.choosing.policy is set to
|
||||
org.apache.hadoop.hdfs.server.datanode.fsdataset.AvailableSpaceVolumeChoosingPolicy.
|
||||
This setting controls how much DN volumes are allowed to differ in terms of
|
||||
bytes of free disk space before they are considered imbalanced. If the free
|
||||
space of all the volumes are within this range of each other, the volumes
|
||||
will be considered balanced and block assignments will be done on a pure
|
||||
round robin basis.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.datanode.fsdataset.volume.choosing.balanced-space-preference-percent</name>
|
||||
<value>0.75f</value>
|
||||
<description>
|
||||
Only used when the dfs.datanode.fsdataset.volume.choosing.policy is set to
|
||||
org.apache.hadoop.hdfs.server.datanode.fsdataset.AvailableSpaceVolumeChoosingPolicy.
|
||||
This setting controls what percentage of new block allocations will be sent
|
||||
to volumes with more available disk space than others. This setting should
|
||||
be in the range 0.0 - 1.0, though in practice 0.5 - 1.0, since there should
|
||||
be no reason to prefer that volumes with less available disk space receive
|
||||
more block allocations.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
</configuration>
|
||||
|
|
|
@ -762,4 +762,27 @@ public class TestDistributedFileSystem {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testFileCloseStatus() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
try {
|
||||
// create a new file.
|
||||
Path file = new Path("/simpleFlush.dat");
|
||||
FSDataOutputStream output = fs.create(file);
|
||||
// write to file
|
||||
output.writeBytes("Some test data");
|
||||
output.flush();
|
||||
assertFalse("File status should be open", fs.isFileClosed(file));
|
||||
output.close();
|
||||
assertTrue("File status should be closed", fs.isFileClosed(file));
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ public class TestHDFSFileSystemContract extends FileSystemContractBaseTest {
|
|||
protected void tearDown() throws Exception {
|
||||
super.tearDown();
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -99,7 +99,7 @@ public class TestSafeMode {
|
|||
*/
|
||||
@Test
|
||||
public void testManualSafeMode() throws IOException {
|
||||
fs = (DistributedFileSystem)cluster.getFileSystem();
|
||||
fs = cluster.getFileSystem();
|
||||
Path file1 = new Path("/tmp/testManualSafeMode/file1");
|
||||
Path file2 = new Path("/tmp/testManualSafeMode/file2");
|
||||
|
||||
|
@ -112,7 +112,7 @@ public class TestSafeMode {
|
|||
// now bring up just the NameNode.
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
|
||||
cluster.waitActive();
|
||||
dfs = (DistributedFileSystem)cluster.getFileSystem();
|
||||
dfs = cluster.getFileSystem();
|
||||
|
||||
assertTrue("No datanode is started. Should be in SafeMode",
|
||||
dfs.setSafeMode(SafeModeAction.SAFEMODE_GET));
|
||||
|
@ -322,11 +322,11 @@ public class TestSafeMode {
|
|||
fs.rename(file1, new Path("file2"));
|
||||
}});
|
||||
|
||||
try {
|
||||
runFsFun("Set time while in SM", new FSRun() {
|
||||
@Override
|
||||
public void run(FileSystem fs) throws IOException {
|
||||
fs.setTimes(file1, 0, 0);
|
||||
} catch (IOException ioe) {
|
||||
fail("Set times failed while in SM");
|
||||
}
|
||||
}});
|
||||
|
||||
try {
|
||||
DFSTestUtil.readFile(fs, file1);
|
||||
|
@ -350,7 +350,7 @@ public class TestSafeMode {
|
|||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
|
||||
|
||||
cluster.restartNameNode();
|
||||
fs = (DistributedFileSystem)cluster.getFileSystem();
|
||||
fs = cluster.getFileSystem();
|
||||
|
||||
String tipMsg = cluster.getNamesystem().getSafemode();
|
||||
assertTrue("Safemode tip message looks right: " + tipMsg,
|
||||
|
@ -375,7 +375,7 @@ public class TestSafeMode {
|
|||
* @throws IOException when there's an issue connecting to the test DFS.
|
||||
*/
|
||||
public void testSafeModeUtils() throws IOException {
|
||||
dfs = (DistributedFileSystem)cluster.getFileSystem();
|
||||
dfs = cluster.getFileSystem();
|
||||
|
||||
// Enter safemode.
|
||||
dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.net.InetSocketAddress;
|
|||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
|
@ -36,8 +37,11 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||
import org.apache.hadoop.test.MockitoUtil;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
/**
|
||||
* This class tests the access time on files.
|
||||
|
@ -274,6 +278,37 @@ public class TestSetTimes {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that when access time updates are not needed, the FSNamesystem
|
||||
* write lock is not taken by getBlockLocations.
|
||||
* Regression test for HDFS-3981.
|
||||
*/
|
||||
@Test(timeout=60000)
|
||||
public void testGetBlockLocationsOnlyUsesReadLock() throws IOException {
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 100*1000);
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
||||
.numDataNodes(0)
|
||||
.build();
|
||||
ReentrantReadWriteLock spyLock = NameNodeAdapter.spyOnFsLock(cluster.getNamesystem());
|
||||
try {
|
||||
// Create empty file in the FSN.
|
||||
Path p = new Path("/empty-file");
|
||||
DFSTestUtil.createFile(cluster.getFileSystem(), p, 0, (short)1, 0L);
|
||||
|
||||
// getBlockLocations() should not need the write lock, since we just created
|
||||
// the file (and thus its access time is already within the 100-second
|
||||
// accesstime precision configured above).
|
||||
MockitoUtil.doThrowWhenCallStackMatches(
|
||||
new AssertionError("Should not need write lock"),
|
||||
".*getBlockLocations.*")
|
||||
.when(spyLock).writeLock();
|
||||
cluster.getFileSystem().getFileBlockLocations(p, 0, 100);
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
new TestSetTimes().testTimes();
|
||||
}
|
||||
|
|
|
@ -98,9 +98,6 @@ public class TestNNWithQJM {
|
|||
|
||||
@Test (timeout = 30000)
|
||||
public void testNewNamenodeTakesOverWriter() throws Exception {
|
||||
// Skip the test on Windows. See HDFS-4584.
|
||||
assumeTrue(!Path.WINDOWS);
|
||||
|
||||
File nn1Dir = new File(
|
||||
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
|
||||
File nn2Dir = new File(
|
||||
|
@ -111,15 +108,19 @@ public class TestNNWithQJM {
|
|||
conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
|
||||
mjc.getQuorumJournalURI("myjournal").toString());
|
||||
|
||||
// Start the cluster once to generate the dfs dirs
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
||||
.numDataNodes(0)
|
||||
.manageNameDfsDirs(false)
|
||||
.checkExitOnShutdown(false)
|
||||
.build();
|
||||
|
||||
try {
|
||||
cluster.getFileSystem().mkdirs(TEST_PATH);
|
||||
// Shutdown the cluster before making a copy of the namenode dir
|
||||
// to release all file locks, otherwise, the copy will fail on
|
||||
// some platforms.
|
||||
cluster.shutdown();
|
||||
|
||||
try {
|
||||
// Start a second NN pointed to the same quorum.
|
||||
// We need to copy the image dir from the first NN -- or else
|
||||
// the new NN will just be rejected because of Namespace mismatch.
|
||||
|
@ -127,6 +128,16 @@ public class TestNNWithQJM {
|
|||
FileUtil.copy(nn1Dir, FileSystem.getLocal(conf).getRaw(),
|
||||
new Path(nn2Dir.getAbsolutePath()), false, conf);
|
||||
|
||||
// Start the cluster again
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.numDataNodes(0)
|
||||
.format(false)
|
||||
.manageNameDfsDirs(false)
|
||||
.checkExitOnShutdown(false)
|
||||
.build();
|
||||
|
||||
cluster.getFileSystem().mkdirs(TEST_PATH);
|
||||
|
||||
Configuration conf2 = new Configuration();
|
||||
conf2.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
|
||||
nn2Dir.getAbsolutePath());
|
||||
|
|
|
@ -75,6 +75,8 @@ public class TestQuorumJournalManager {
|
|||
private QuorumJournalManager qjm;
|
||||
private List<AsyncLogger> spies;
|
||||
|
||||
private List<QuorumJournalManager> toClose = Lists.newLinkedList();
|
||||
|
||||
static {
|
||||
((Log4JLogger)ProtobufRpcEngine.LOG).getLogger().setLevel(Level.ALL);
|
||||
}
|
||||
|
@ -98,11 +100,26 @@ public class TestQuorumJournalManager {
|
|||
|
||||
@After
|
||||
public void shutdown() throws IOException {
|
||||
IOUtils.cleanup(LOG, toClose.toArray(new Closeable[0]));
|
||||
|
||||
// Should not leak clients between tests -- this can cause flaky tests.
|
||||
// (See HDFS-4643)
|
||||
GenericTestUtils.assertNoThreadsMatching(".*IPC Client.*");
|
||||
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Enqueue a QJM for closing during shutdown. This makes the code a little
|
||||
* easier to follow, with fewer try..finally clauses necessary.
|
||||
*/
|
||||
private QuorumJournalManager closeLater(QuorumJournalManager qjm) {
|
||||
toClose.add(qjm);
|
||||
return qjm;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleWriter() throws Exception {
|
||||
writeSegment(cluster, qjm, 1, 3, true);
|
||||
|
@ -119,8 +136,8 @@ public class TestQuorumJournalManager {
|
|||
|
||||
@Test
|
||||
public void testFormat() throws Exception {
|
||||
QuorumJournalManager qjm = new QuorumJournalManager(
|
||||
conf, cluster.getQuorumJournalURI("testFormat-jid"), FAKE_NSINFO);
|
||||
QuorumJournalManager qjm = closeLater(new QuorumJournalManager(
|
||||
conf, cluster.getQuorumJournalURI("testFormat-jid"), FAKE_NSINFO));
|
||||
assertFalse(qjm.hasSomeData());
|
||||
qjm.format(FAKE_NSINFO);
|
||||
assertTrue(qjm.hasSomeData());
|
||||
|
@ -128,8 +145,7 @@ public class TestQuorumJournalManager {
|
|||
|
||||
@Test
|
||||
public void testReaderWhileAnotherWrites() throws Exception {
|
||||
|
||||
QuorumJournalManager readerQjm = createSpyingQJM();
|
||||
QuorumJournalManager readerQjm = closeLater(createSpyingQJM());
|
||||
List<EditLogInputStream> streams = Lists.newArrayList();
|
||||
readerQjm.selectInputStreams(streams, 0, false);
|
||||
assertEquals(0, streams.size());
|
||||
|
@ -251,8 +267,8 @@ public class TestQuorumJournalManager {
|
|||
|
||||
|
||||
// Make a new QJM
|
||||
qjm = new QuorumJournalManager(
|
||||
conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO);
|
||||
qjm = closeLater(new QuorumJournalManager(
|
||||
conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO));
|
||||
qjm.recoverUnfinalizedSegments();
|
||||
checkRecovery(cluster, 1, 3);
|
||||
|
||||
|
@ -364,8 +380,8 @@ public class TestQuorumJournalManager {
|
|||
NNStorage.getInProgressEditsFileName(1));
|
||||
|
||||
// Make a new QJM
|
||||
qjm = new QuorumJournalManager(
|
||||
conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO);
|
||||
qjm = closeLater(new QuorumJournalManager(
|
||||
conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO));
|
||||
qjm.recoverUnfinalizedSegments();
|
||||
checkRecovery(cluster, 1, 3);
|
||||
}
|
||||
|
@ -902,8 +918,8 @@ public class TestQuorumJournalManager {
|
|||
return Mockito.spy(logger);
|
||||
}
|
||||
};
|
||||
return new QuorumJournalManager(
|
||||
conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO, spyFactory);
|
||||
return closeLater(new QuorumJournalManager(
|
||||
conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO, spyFactory));
|
||||
}
|
||||
|
||||
private static void waitForAllPendingCalls(AsyncLoggerSet als)
|
||||
|
|
|
@ -17,10 +17,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.server.datanode;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertSame;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Map;
|
||||
|
@ -68,6 +70,8 @@ public class TestBPOfferService {
|
|||
TestBPOfferService.class);
|
||||
private static final ExtendedBlock FAKE_BLOCK =
|
||||
new ExtendedBlock(FAKE_BPID, 12345L);
|
||||
private static final String TEST_BUILD_DATA = System.getProperty(
|
||||
"test.build.data", "build/test/data");
|
||||
|
||||
static {
|
||||
((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
|
||||
|
@ -90,6 +94,8 @@ public class TestBPOfferService {
|
|||
mockDn = Mockito.mock(DataNode.class);
|
||||
Mockito.doReturn(true).when(mockDn).shouldRun();
|
||||
Configuration conf = new Configuration();
|
||||
File dnDataDir = new File(new File(TEST_BUILD_DATA, "dfs"), "data");
|
||||
conf.set(DFS_DATANODE_DATA_DIR_KEY, dnDataDir.toURI().toString());
|
||||
Mockito.doReturn(conf).when(mockDn).getConf();
|
||||
Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
|
||||
Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn"))
|
||||
|
|
|
@ -113,7 +113,7 @@ public class TestBlockPoolManager {
|
|||
|
||||
// Remove the first NS
|
||||
conf.set(DFSConfigKeys.DFS_NAMESERVICES,
|
||||
"ns1");
|
||||
"ns2");
|
||||
bpm.refreshNamenodes(conf);
|
||||
assertEquals(
|
||||
"stop #1\n" +
|
||||
|
|
|
@ -0,0 +1,303 @@
|
|||
/**
|
||||
* 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.fsdataset;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class TestAvailableSpaceVolumeChoosingPolicy {
|
||||
|
||||
private static final int RANDOMIZED_ITERATIONS = 10000;
|
||||
private static final float RANDOMIZED_ERROR_PERCENT = 0.05f;
|
||||
private static final long RANDOMIZED_ALLOWED_ERROR = (long) (RANDOMIZED_ERROR_PERCENT * RANDOMIZED_ITERATIONS);
|
||||
|
||||
private static void initPolicy(VolumeChoosingPolicy<FsVolumeSpi> policy,
|
||||
float preferencePercent) {
|
||||
Configuration conf = new Configuration();
|
||||
// Set the threshold to consider volumes imbalanced to 1MB
|
||||
conf.setLong(
|
||||
DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_THRESHOLD_KEY,
|
||||
1024 * 1024); // 1MB
|
||||
conf.setFloat(
|
||||
DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_BALANCED_SPACE_PREFERENCE_PERCENT_KEY,
|
||||
preferencePercent);
|
||||
((Configurable) policy).setConf(conf);
|
||||
}
|
||||
|
||||
// Test the Round-Robin block-volume fallback path when all volumes are within
|
||||
// the threshold.
|
||||
@Test(timeout=60000)
|
||||
public void testRR() throws Exception {
|
||||
@SuppressWarnings("unchecked")
|
||||
final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
|
||||
initPolicy(policy, 1.0f);
|
||||
TestRoundRobinVolumeChoosingPolicy.testRR(policy);
|
||||
}
|
||||
|
||||
// ChooseVolume should throw DiskOutOfSpaceException
|
||||
// with volume and block sizes in exception message.
|
||||
@Test(timeout=60000)
|
||||
public void testRRPolicyExceptionMessage() throws Exception {
|
||||
final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy
|
||||
= new AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi>();
|
||||
initPolicy(policy, 1.0f);
|
||||
TestRoundRobinVolumeChoosingPolicy.testRRPolicyExceptionMessage(policy);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testTwoUnbalancedVolumes() throws Exception {
|
||||
@SuppressWarnings("unchecked")
|
||||
final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
|
||||
initPolicy(policy, 1.0f);
|
||||
|
||||
List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
|
||||
|
||||
// First volume with 1MB free space
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
|
||||
|
||||
// Second volume with 3MB free space, which is a difference of 2MB, more
|
||||
// than the threshold of 1MB.
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(1).getAvailable()).thenReturn(1024L * 1024L * 3);
|
||||
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testThreeUnbalancedVolumes() throws Exception {
|
||||
@SuppressWarnings("unchecked")
|
||||
final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
|
||||
|
||||
List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
|
||||
|
||||
// First volume with 1MB free space
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
|
||||
|
||||
// Second volume with 3MB free space, which is a difference of 2MB, more
|
||||
// than the threshold of 1MB.
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(1).getAvailable()).thenReturn(1024L * 1024L * 3);
|
||||
|
||||
// Third volume, again with 3MB free space.
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(2).getAvailable()).thenReturn(1024L * 1024L * 3);
|
||||
|
||||
// We should alternate assigning between the two volumes with a lot of free
|
||||
// space.
|
||||
initPolicy(policy, 1.0f);
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(2), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(2), policy.chooseVolume(volumes, 100));
|
||||
|
||||
// All writes should be assigned to the volume with the least free space.
|
||||
initPolicy(policy, 0.0f);
|
||||
Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testFourUnbalancedVolumes() throws Exception {
|
||||
@SuppressWarnings("unchecked")
|
||||
final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
|
||||
|
||||
List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
|
||||
|
||||
// First volume with 1MB free space
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
|
||||
|
||||
// Second volume with 1MB + 1 byte free space
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(1).getAvailable()).thenReturn(1024L * 1024L + 1);
|
||||
|
||||
// Third volume with 3MB free space, which is a difference of 2MB, more
|
||||
// than the threshold of 1MB.
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(2).getAvailable()).thenReturn(1024L * 1024L * 3);
|
||||
|
||||
// Fourth volume, again with 3MB free space.
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(3).getAvailable()).thenReturn(1024L * 1024L * 3);
|
||||
|
||||
// We should alternate assigning between the two volumes with a lot of free
|
||||
// space.
|
||||
initPolicy(policy, 1.0f);
|
||||
Assert.assertEquals(volumes.get(2), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(3), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(2), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(3), policy.chooseVolume(volumes, 100));
|
||||
|
||||
// We should alternate assigning between the two volumes with less free
|
||||
// space.
|
||||
initPolicy(policy, 0.0f);
|
||||
Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 100));
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testNotEnoughSpaceOnSelectedVolume() throws Exception {
|
||||
@SuppressWarnings("unchecked")
|
||||
final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
|
||||
|
||||
List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
|
||||
|
||||
// First volume with 1MB free space
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
|
||||
|
||||
// Second volume with 3MB free space, which is a difference of 2MB, more
|
||||
// than the threshold of 1MB.
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(1).getAvailable()).thenReturn(1024L * 1024L * 3);
|
||||
|
||||
// All writes should be assigned to the volume with the least free space.
|
||||
// However, if the volume with the least free space doesn't have enough
|
||||
// space to accept the replica size, and another volume does have enough
|
||||
// free space, that should be chosen instead.
|
||||
initPolicy(policy, 0.0f);
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 1024L * 1024L * 2));
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testAvailableSpaceChanges() throws Exception {
|
||||
@SuppressWarnings("unchecked")
|
||||
final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
|
||||
initPolicy(policy, 1.0f);
|
||||
|
||||
List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
|
||||
|
||||
// First volume with 1MB free space
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(0).getAvailable()).thenReturn(1024L * 1024L);
|
||||
|
||||
// Second volume with 3MB free space, which is a difference of 2MB, more
|
||||
// than the threshold of 1MB.
|
||||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(1).getAvailable())
|
||||
.thenReturn(1024L * 1024L * 3)
|
||||
.thenReturn(1024L * 1024L * 3)
|
||||
.thenReturn(1024L * 1024L * 3)
|
||||
.thenReturn(1024L * 1024L * 1); // After the third check, return 1MB.
|
||||
|
||||
// Should still be able to get a volume for the replica even though the
|
||||
// available space on the second volume changed.
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 100));
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void randomizedTest1() throws Exception {
|
||||
doRandomizedTest(0.75f, 1, 1);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void randomizedTest2() throws Exception {
|
||||
doRandomizedTest(0.75f, 5, 1);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void randomizedTest3() throws Exception {
|
||||
doRandomizedTest(0.75f, 1, 5);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void randomizedTest4() throws Exception {
|
||||
doRandomizedTest(0.90f, 5, 1);
|
||||
}
|
||||
|
||||
/*
|
||||
* Ensure that we randomly select the lesser-used volumes with appropriate
|
||||
* frequency.
|
||||
*/
|
||||
public void doRandomizedTest(float preferencePercent, int lowSpaceVolumes,
|
||||
int highSpaceVolumes) throws Exception {
|
||||
@SuppressWarnings("unchecked")
|
||||
final AvailableSpaceVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(AvailableSpaceVolumeChoosingPolicy.class, null);
|
||||
|
||||
List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
|
||||
|
||||
// Volumes with 1MB free space
|
||||
for (int i = 0; i < lowSpaceVolumes; i++) {
|
||||
FsVolumeSpi volume = Mockito.mock(FsVolumeSpi.class);
|
||||
Mockito.when(volume.getAvailable()).thenReturn(1024L * 1024L);
|
||||
volumes.add(volume);
|
||||
}
|
||||
|
||||
// Volumes with 3MB free space
|
||||
for (int i = 0; i < highSpaceVolumes; i++) {
|
||||
FsVolumeSpi volume = Mockito.mock(FsVolumeSpi.class);
|
||||
Mockito.when(volume.getAvailable()).thenReturn(1024L * 1024L * 3);
|
||||
volumes.add(volume);
|
||||
}
|
||||
|
||||
initPolicy(policy, preferencePercent);
|
||||
long lowAvailableSpaceVolumeSelected = 0;
|
||||
long highAvailableSpaceVolumeSelected = 0;
|
||||
for (int i = 0; i < RANDOMIZED_ITERATIONS; i++) {
|
||||
FsVolumeSpi volume = policy.chooseVolume(volumes, 100);
|
||||
for (int j = 0; j < volumes.size(); j++) {
|
||||
// Note how many times the first low available volume was selected
|
||||
if (volume == volumes.get(j) && j == 0) {
|
||||
lowAvailableSpaceVolumeSelected++;
|
||||
}
|
||||
// Note how many times the first high available volume was selected
|
||||
if (volume == volumes.get(j) && j == lowSpaceVolumes) {
|
||||
highAvailableSpaceVolumeSelected++;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Calculate the expected ratio of how often low available space volumes
|
||||
// were selected vs. high available space volumes.
|
||||
float expectedSelectionRatio = preferencePercent / (1 - preferencePercent);
|
||||
|
||||
GenericTestUtils.assertValueNear(
|
||||
(long)(lowAvailableSpaceVolumeSelected * expectedSelectionRatio),
|
||||
highAvailableSpaceVolumeSelected,
|
||||
RANDOMIZED_ALLOWED_ERROR);
|
||||
}
|
||||
|
||||
}
|
|
@ -32,6 +32,14 @@ public class TestRoundRobinVolumeChoosingPolicy {
|
|||
// Test the Round-Robin block-volume choosing algorithm.
|
||||
@Test
|
||||
public void testRR() throws Exception {
|
||||
@SuppressWarnings("unchecked")
|
||||
final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(RoundRobinVolumeChoosingPolicy.class, null);
|
||||
testRR(policy);
|
||||
}
|
||||
|
||||
public static void testRR(VolumeChoosingPolicy<FsVolumeSpi> policy)
|
||||
throws Exception {
|
||||
final List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
|
||||
|
||||
// First volume, with 100 bytes of space.
|
||||
|
@ -42,10 +50,6 @@ public class TestRoundRobinVolumeChoosingPolicy {
|
|||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(1).getAvailable()).thenReturn(200L);
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy =
|
||||
ReflectionUtils.newInstance(RoundRobinVolumeChoosingPolicy.class, null);
|
||||
|
||||
// Test two rounds of round-robin choosing
|
||||
Assert.assertEquals(volumes.get(0), policy.chooseVolume(volumes, 0));
|
||||
Assert.assertEquals(volumes.get(1), policy.chooseVolume(volumes, 0));
|
||||
|
@ -69,6 +73,13 @@ public class TestRoundRobinVolumeChoosingPolicy {
|
|||
// with volume and block sizes in exception message.
|
||||
@Test
|
||||
public void testRRPolicyExceptionMessage() throws Exception {
|
||||
final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy
|
||||
= new RoundRobinVolumeChoosingPolicy<FsVolumeSpi>();
|
||||
testRRPolicyExceptionMessage(policy);
|
||||
}
|
||||
|
||||
public static void testRRPolicyExceptionMessage(
|
||||
VolumeChoosingPolicy<FsVolumeSpi> policy) throws Exception {
|
||||
final List<FsVolumeSpi> volumes = new ArrayList<FsVolumeSpi>();
|
||||
|
||||
// First volume, with 500 bytes of space.
|
||||
|
@ -79,8 +90,6 @@ public class TestRoundRobinVolumeChoosingPolicy {
|
|||
volumes.add(Mockito.mock(FsVolumeSpi.class));
|
||||
Mockito.when(volumes.get(1).getAvailable()).thenReturn(600L);
|
||||
|
||||
final RoundRobinVolumeChoosingPolicy<FsVolumeSpi> policy
|
||||
= new RoundRobinVolumeChoosingPolicy<FsVolumeSpi>();
|
||||
int blockSize = 700;
|
||||
try {
|
||||
policy.chooseVolume(volumes, blockSize);
|
||||
|
|
|
@ -217,7 +217,8 @@ public abstract class FSImageTestUtil {
|
|||
FsPermission.createImmutable((short)0755));
|
||||
for (int i = 1; i <= numDirs; i++) {
|
||||
String dirName = "dir" + i;
|
||||
INodeDirectory dir = new INodeDirectory(newInodeId + i -1, dirName, perms);
|
||||
INodeDirectory dir = new INodeDirectory(newInodeId + i - 1, dirName,
|
||||
perms);
|
||||
editLog.logMkDir("/" + dirName, dir);
|
||||
}
|
||||
editLog.logSync();
|
||||
|
|
|
@ -30,8 +30,6 @@ import static org.junit.Assert.fail;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.lang.management.ThreadInfo;
|
||||
import java.lang.management.ThreadMXBean;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
|
@ -122,18 +120,7 @@ public class TestCheckpoint {
|
|||
|
||||
@After
|
||||
public void checkForSNNThreads() {
|
||||
ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
|
||||
|
||||
ThreadInfo[] infos = threadBean.getThreadInfo(threadBean.getAllThreadIds(), 20);
|
||||
for (ThreadInfo info : infos) {
|
||||
if (info == null) continue;
|
||||
LOG.info("Check thread: " + info.getThreadName());
|
||||
if (info.getThreadName().contains("SecondaryNameNode")) {
|
||||
fail("Leaked thread: " + info + "\n" +
|
||||
Joiner.on("\n").join(info.getStackTrace()));
|
||||
}
|
||||
}
|
||||
LOG.info("--------");
|
||||
GenericTestUtils.assertNoThreadsMatching(".*SecondaryNameNode.*");
|
||||
}
|
||||
|
||||
static void checkFile(FileSystem fileSys, Path name, int repl)
|
||||
|
|
|
@ -24,10 +24,8 @@ import static org.junit.Assert.fail;
|
|||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Options;
|
||||
|
@ -39,10 +37,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||
import org.apache.hadoop.io.EnumSetWritable;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestINodeFile {
|
||||
|
@ -398,9 +393,7 @@ public class TestINodeFile {
|
|||
cluster.waitActive();
|
||||
|
||||
FSNamesystem fsn = cluster.getNamesystem();
|
||||
long lastId = fsn.getLastInodeId();
|
||||
|
||||
assertTrue(lastId == 1001);
|
||||
assertTrue(fsn.getLastInodeId() == 1001);
|
||||
|
||||
// Create one directory and the last inode id should increase to 1002
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
|
@ -408,14 +401,10 @@ public class TestINodeFile {
|
|||
assertTrue(fs.mkdirs(path));
|
||||
assertTrue(fsn.getLastInodeId() == 1002);
|
||||
|
||||
// Use namenode rpc to create a file
|
||||
NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
|
||||
HdfsFileStatus fileStatus = nnrpc.create("/test1/file", new FsPermission(
|
||||
(short) 0755), "client",
|
||||
new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)), true,
|
||||
(short) 1, 128 * 1024 * 1024L);
|
||||
int fileLen = 1024;
|
||||
Path filePath = new Path("/test1/file");
|
||||
DFSTestUtil.createFile(fs, filePath, fileLen, (short) 1, 0);
|
||||
assertTrue(fsn.getLastInodeId() == 1003);
|
||||
assertTrue(fileStatus.getFileId() == 1003);
|
||||
|
||||
// Rename doesn't increase inode id
|
||||
Path renamedPath = new Path("/test2");
|
||||
|
@ -427,7 +416,32 @@ public class TestINodeFile {
|
|||
// Make sure empty editlog can be handled
|
||||
cluster.restartNameNode();
|
||||
cluster.waitActive();
|
||||
fsn = cluster.getNamesystem();
|
||||
assertTrue(fsn.getLastInodeId() == 1003);
|
||||
|
||||
DFSTestUtil.createFile(fs, new Path("/test2/file2"), fileLen, (short) 1,
|
||||
0);
|
||||
long id = fsn.getLastInodeId();
|
||||
assertTrue(id == 1004);
|
||||
fs.delete(new Path("/test2"), true);
|
||||
// create a file under construction
|
||||
FSDataOutputStream outStream = fs.create(new Path("/test3/file"));
|
||||
assertTrue(outStream != null);
|
||||
assertTrue(fsn.getLastInodeId() == 1006);
|
||||
|
||||
// Apply editlogs to fsimage, test fsimage with inodeUnderConstruction can
|
||||
// be handled
|
||||
fsn.enterSafeMode(false);
|
||||
fsn.saveNamespace();
|
||||
fsn.leaveSafeMode();
|
||||
|
||||
outStream.close();
|
||||
|
||||
// The lastInodeId in fsimage should remain 1006 after reboot
|
||||
cluster.restartNameNode();
|
||||
cluster.waitActive();
|
||||
fsn = cluster.getNamesystem();
|
||||
assertTrue(fsn.getLastInodeId() == 1006);
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.File;
|
||||
|
@ -30,12 +31,14 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.junit.Test;
|
||||
import static org.mockito.Mockito.*;
|
||||
|
||||
/**
|
||||
* This class tests the creation and validation of a checkpoint.
|
||||
|
@ -163,4 +166,70 @@ public class TestSecurityTokenEditLog {
|
|||
if(cluster != null) cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testEditsForCancelOnTokenExpire() throws IOException,
|
||||
InterruptedException {
|
||||
long renewInterval = 2000;
|
||||
Configuration conf = new Configuration();
|
||||
conf.setBoolean(
|
||||
DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
|
||||
conf.setLong(DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, renewInterval);
|
||||
conf.setLong(DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, renewInterval*2);
|
||||
|
||||
Text renewer = new Text(UserGroupInformation.getCurrentUser().getUserName());
|
||||
FSImage fsImage = mock(FSImage.class);
|
||||
FSEditLog log = mock(FSEditLog.class);
|
||||
doReturn(log).when(fsImage).getEditLog();
|
||||
FSNamesystem fsn = new FSNamesystem(conf, fsImage);
|
||||
|
||||
DelegationTokenSecretManager dtsm = fsn.getDelegationTokenSecretManager();
|
||||
try {
|
||||
dtsm.startThreads();
|
||||
|
||||
// get two tokens
|
||||
Token<DelegationTokenIdentifier> token1 = fsn.getDelegationToken(renewer);
|
||||
Token<DelegationTokenIdentifier> token2 = fsn.getDelegationToken(renewer);
|
||||
DelegationTokenIdentifier ident1 =
|
||||
(DelegationTokenIdentifier)token1.decodeIdentifier();
|
||||
DelegationTokenIdentifier ident2 =
|
||||
(DelegationTokenIdentifier)token2.decodeIdentifier();
|
||||
|
||||
// verify we got the tokens
|
||||
verify(log, times(1)).logGetDelegationToken(eq(ident1), anyLong());
|
||||
verify(log, times(1)).logGetDelegationToken(eq(ident2), anyLong());
|
||||
|
||||
// this is a little tricky because DTSM doesn't let us set scan interval
|
||||
// so need to periodically sleep, then stop/start threads to force scan
|
||||
|
||||
// renew first token 1/2 to expire
|
||||
Thread.sleep(renewInterval/2);
|
||||
fsn.renewDelegationToken(token2);
|
||||
verify(log, times(1)).logRenewDelegationToken(eq(ident2), anyLong());
|
||||
// force scan and give it a little time to complete
|
||||
dtsm.stopThreads(); dtsm.startThreads();
|
||||
Thread.sleep(250);
|
||||
// no token has expired yet
|
||||
verify(log, times(0)).logCancelDelegationToken(eq(ident1));
|
||||
verify(log, times(0)).logCancelDelegationToken(eq(ident2));
|
||||
|
||||
// sleep past expiration of 1st non-renewed token
|
||||
Thread.sleep(renewInterval/2);
|
||||
dtsm.stopThreads(); dtsm.startThreads();
|
||||
Thread.sleep(250);
|
||||
// non-renewed token should have implicitly been cancelled
|
||||
verify(log, times(1)).logCancelDelegationToken(eq(ident1));
|
||||
verify(log, times(0)).logCancelDelegationToken(eq(ident2));
|
||||
|
||||
// sleep past expiration of 2nd renewed token
|
||||
Thread.sleep(renewInterval/2);
|
||||
dtsm.stopThreads(); dtsm.startThreads();
|
||||
Thread.sleep(250);
|
||||
// both tokens should have been implicitly cancelled by now
|
||||
verify(log, times(1)).logCancelDelegationToken(eq(ident1));
|
||||
verify(log, times(1)).logCancelDelegationToken(eq(ident2));
|
||||
} finally {
|
||||
dtsm.stopThreads();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -388,6 +388,11 @@ public class TestGSet {
|
|||
return String.format(" iterate=%5d, contain=%5d, time elapsed=%5d.%03ds",
|
||||
iterate_count, contain_count, t/1000, t%1000);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
gset.clear();
|
||||
}
|
||||
}
|
||||
|
||||
/** Test data set */
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
package org.apache.hadoop.hdfs.web;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -29,9 +31,13 @@ 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.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.TestDFSClientRetries;
|
||||
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -208,4 +214,48 @@ public class TestWebHDFS {
|
|||
final Configuration conf = WebHdfsTestUtil.createConf();
|
||||
TestDFSClientRetries.namenodeRestartTest(conf, true);
|
||||
}
|
||||
|
||||
@Test(timeout=300000)
|
||||
public void testLargeDirectory() throws Exception {
|
||||
final Configuration conf = WebHdfsTestUtil.createConf();
|
||||
final int listLimit = 2;
|
||||
// force small chunking of directory listing
|
||||
conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, listLimit);
|
||||
// force paths to be only owner-accessible to ensure ugi isn't changing
|
||||
// during listStatus
|
||||
FsPermission.setUMask(conf, new FsPermission((short)0077));
|
||||
|
||||
final MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
||||
try {
|
||||
cluster.waitActive();
|
||||
WebHdfsTestUtil.getWebHdfsFileSystem(conf).setPermission(
|
||||
new Path("/"),
|
||||
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
|
||||
|
||||
// trick the NN into not believing it's not the superuser so we can
|
||||
// tell if the correct user is used by listStatus
|
||||
UserGroupInformation.setLoginUser(
|
||||
UserGroupInformation.createUserForTesting(
|
||||
"not-superuser", new String[]{"not-supergroup"}));
|
||||
|
||||
UserGroupInformation.createUserForTesting("me", new String[]{"my-group"})
|
||||
.doAs(new PrivilegedExceptionAction<Void>() {
|
||||
@Override
|
||||
public Void run() throws IOException, URISyntaxException {
|
||||
FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf);
|
||||
Path d = new Path("/my-dir");
|
||||
Assert.assertTrue(fs.mkdirs(d));
|
||||
for (int i=0; i < listLimit*3; i++) {
|
||||
Path p = new Path(d, "file-"+i);
|
||||
Assert.assertTrue(fs.createNewFile(p));
|
||||
}
|
||||
Assert.assertEquals(listLimit*3, fs.listStatus(d).length);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Binary file not shown.
|
@ -1,6 +1,6 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<EDITS>
|
||||
<EDITS_VERSION>-40</EDITS_VERSION>
|
||||
<EDITS_VERSION>-42</EDITS_VERSION>
|
||||
<RECORD>
|
||||
<OPCODE>OP_START_LOG_SEGMENT</OPCODE>
|
||||
<DATA>
|
||||
|
@ -41,6 +41,7 @@
|
|||
<DATA>
|
||||
<TXID>5</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>1002</INODEID>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405685834</MTIME>
|
||||
|
@ -60,6 +61,7 @@
|
|||
<DATA>
|
||||
<TXID>6</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405685848</MTIME>
|
||||
|
@ -98,6 +100,7 @@
|
|||
<DATA>
|
||||
<TXID>9</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>1003</INODEID>
|
||||
<PATH>/directory_mkdir</PATH>
|
||||
<TIMESTAMP>1330405685861</TIMESTAMP>
|
||||
<PERMISSION_STATUS>
|
||||
|
@ -119,6 +122,7 @@
|
|||
<DATA>
|
||||
<TXID>11</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>1004</INODEID>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405685866</MTIME>
|
||||
|
@ -138,6 +142,7 @@
|
|||
<DATA>
|
||||
<TXID>12</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405685868</MTIME>
|
||||
|
@ -218,6 +223,7 @@
|
|||
<DATA>
|
||||
<TXID>20</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>1005</INODEID>
|
||||
<PATH>/file_concat_target</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405685889</MTIME>
|
||||
|
@ -309,6 +315,7 @@
|
|||
<DATA>
|
||||
<TXID>27</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/file_concat_target</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405685978</MTIME>
|
||||
|
@ -350,6 +357,7 @@
|
|||
<DATA>
|
||||
<TXID>29</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>1006</INODEID>
|
||||
<PATH>/file_concat_0</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405685983</MTIME>
|
||||
|
@ -441,6 +449,7 @@
|
|||
<DATA>
|
||||
<TXID>36</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/file_concat_0</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405686013</MTIME>
|
||||
|
@ -482,6 +491,7 @@
|
|||
<DATA>
|
||||
<TXID>38</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>1007</INODEID>
|
||||
<PATH>/file_concat_1</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405686017</MTIME>
|
||||
|
@ -573,6 +583,7 @@
|
|||
<DATA>
|
||||
<TXID>45</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/file_concat_1</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405686042</MTIME>
|
||||
|
@ -620,6 +631,7 @@
|
|||
<DATA>
|
||||
<TXID>47</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>1008</INODEID>
|
||||
<PATH>/file_symlink</PATH>
|
||||
<VALUE>/file_concat_target</VALUE>
|
||||
<MTIME>1330405686051</MTIME>
|
||||
|
@ -693,6 +705,7 @@
|
|||
<DATA>
|
||||
<TXID>52</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>1009</INODEID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405686084</MTIME>
|
||||
|
@ -759,6 +772,7 @@
|
|||
<DATA>
|
||||
<TXID>58</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1330405688726</MTIME>
|
||||
|
|
|
@ -14,10 +14,11 @@ Trunk (Unreleased)
|
|||
MAPREDUCE-4887. Add RehashPartitioner, to smooth distributions
|
||||
with poor implementations of Object#hashCode(). (Radim Kolar via cutting)
|
||||
|
||||
IMPROVEMENTS
|
||||
HADOOP-8562. Enhancements to support Hadoop on Windows Server and Windows
|
||||
Azure environments. (See breakdown of tasks below for subtasks and
|
||||
contributors)
|
||||
|
||||
MAPREDUCE-3787. [Gridmix] Optimize job monitoring and STRESS mode for
|
||||
faster job submission. (amarrk)
|
||||
IMPROVEMENTS
|
||||
|
||||
MAPREDUCE-3481. [Gridmix] Improve Gridmix STRESS mode. (amarrk)
|
||||
|
||||
|
@ -30,9 +31,6 @@ Trunk (Unreleased)
|
|||
MAPREDUCE-2733. [Gridmix] Gridmix3 cpu emulation system tests.
|
||||
(Vinay Thota via amarrk)
|
||||
|
||||
MAPREDUCE-3008. Improvements to cumulative CPU emulation for short running
|
||||
tasks in Gridmix. (amarrk)
|
||||
|
||||
MAPREDUCE-2836. Provide option to fail jobs when submitted to non-existent
|
||||
fair scheduler pools. (Ahmed Radwan via todd)
|
||||
|
||||
|
@ -71,39 +69,14 @@ Trunk (Unreleased)
|
|||
MAPREDUCE-4735. Make arguments in TestDFSIO case insensitive.
|
||||
(Brandon Li via suresh)
|
||||
|
||||
MAPREDUCE-5014. Extend Distcp to accept a custom CopyListing.
|
||||
(Srikanth Sundarrajan via amareshwari)
|
||||
|
||||
BUG FIXES
|
||||
|
||||
MAPREDUCE-4272. SortedRanges.Range#compareTo is not spec compliant.
|
||||
(Yu Gao via llu)
|
||||
|
||||
MAPREDUCE-4356. [Rumen] Provide access to the method
|
||||
ParsedTask.obtainTaskAttempts(). (ravigummadi)
|
||||
|
||||
MAPREDUCE-4100. [Gridmix] Bug fixed in compression emulation feature for
|
||||
map only jobs. (amarrk)
|
||||
|
||||
MAPREDUCE-4149. [Rumen] Rumen fails to parse certain counter
|
||||
strings. (ravigummadi)
|
||||
|
||||
MAPREDUCE-4083. [Gridmix] NPE in cpu emulation. (amarrk)
|
||||
|
||||
MAPREDUCE-4087. [Gridmix] GenerateDistCacheData job of Gridmix can
|
||||
become slow in some cases (ravigummadi).
|
||||
|
||||
MAPREDUCE-3953. [Gridmix] Gridmix throws NPE and does not simulate a
|
||||
job if the trace contains null taskStatus for a task.
|
||||
(ravigummadi)
|
||||
|
||||
MAPREDUCE-3829. [Gridmix] Gridmix should give better error message when
|
||||
input data directory already exists and -generate opton is
|
||||
given.(ravigummadi)
|
||||
|
||||
MAPREDUCE-2722. [Gridmix] Gridmix simulated job's map's hdfsBytesRead
|
||||
counter is wrong when compressed input is used.(ravigummadi)
|
||||
|
||||
MAPREDUCE-3757. [Rumen] Fixed Rumen Folder to adjust shuffleFinished and
|
||||
sortFinished times when needed.(ravigummadi)
|
||||
|
||||
MAPREDUCE-3194. "mapred mradmin" command is broken in mrv2
|
||||
(Jason Lowe via bobby)
|
||||
|
||||
|
@ -155,10 +128,34 @@ Trunk (Unreleased)
|
|||
MAPREDUCE-5012. Typo in javadoc for IdentityMapper class. (Adam Monsen
|
||||
via suresh)
|
||||
|
||||
MAPREDUCE-5006. Fix failing streaming tests due to MAPREDUCE-4994.
|
||||
(Sandy Ryza via tomwhite)
|
||||
MAPREDUCE-5078. TestMRAppMaster fails on Windows due to mismatched path
|
||||
separators. (Chris Nauroth via sseth)
|
||||
|
||||
Release 2.0.4-beta - UNRELEASED
|
||||
MAPREDUCE-4885. Streaming tests have multiple failures on Windows. (Chris
|
||||
Nauroth via bikas)
|
||||
|
||||
BREAKDOWN OF HADOOP-8562 SUBTASKS
|
||||
|
||||
MAPREDUCE-4739. Some MapReduce tests fail to find winutils.
|
||||
(Chris Nauroth via suresh)
|
||||
|
||||
MAPREDUCE-4780. MapReduce distribution build fails on Windows.
|
||||
(Chris Nauroth via suresh)
|
||||
|
||||
MAPREDUCE-4790. MapReduce build script would be more readable using abspath.
|
||||
(Chris Nauroth via suresh)
|
||||
|
||||
MAPREDUCE-4869. Fix TestMapReduceChildJVM. (Chris Nauroth via acmurthy)
|
||||
|
||||
MAPREDUCE-4870. Fix TestMRJobsWithHistoryService. (Chris Nauroth via acmurthy)
|
||||
|
||||
MAPREDUCE-4983. Fixed various platform specific assumptions in various tests,
|
||||
so that they can pass on Windows too. (Chris Nauroth via vinodkv)
|
||||
|
||||
HADOOP-9372. Fix bad timeout annotations on tests.
|
||||
(Arpit Agarwal via suresh)
|
||||
|
||||
Release 2.0.5-beta - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
|
@ -166,9 +163,41 @@ Release 2.0.4-beta - UNRELEASED
|
|||
|
||||
IMPROVEMENTS
|
||||
|
||||
MAPREDUCE-3008. Improvements to cumulative CPU emulation for short running
|
||||
tasks in Gridmix. (amarrk via tgraves)
|
||||
|
||||
MAPREDUCE-5033. mapred shell script should respect usage flags
|
||||
(--help -help -h). (Andrew Wang via atm)
|
||||
|
||||
MAPREDUCE-4892. Modify CombineFileInputFormat to not skew input slits'
|
||||
allocation on small clusters. (Bikas Saha via vinodkv)
|
||||
|
||||
MAPREDUCE-4990. Construct debug strings conditionally in
|
||||
ShuffleHandler.Shuffle#sendMapOutput(). (kkambatl via tucu)
|
||||
|
||||
MAPREDUCE-4875. coverage fixing for org.apache.hadoop.mapred
|
||||
(Aleksey Gorshkov via bobby)
|
||||
|
||||
MAPREDUCE-5129. Allow tags to JobHistory for deeper analytics. (billie via
|
||||
acmurthy)
|
||||
|
||||
MAPREDUCE-3787. [Gridmix] Optimize job monitoring and STRESS mode for
|
||||
faster job submission. (amarrk via tgraves)
|
||||
|
||||
MAPREDUCE-5079. Changes job recovery to restore state directly from job
|
||||
history, instaed of simulating state machine events.
|
||||
(Jason Lowe and Robert Parker via sseth)
|
||||
|
||||
MAPREDUCE-4981. Add WordMean, WordMedian, WordStandardDeviation
|
||||
to ExamplesDriver. (Plamen Jeliazkov via shv)
|
||||
|
||||
MAPREUDUCE-5059. Change average merge time on Job overview page to be the
|
||||
time delta between the end of the shuffle and the start of the reduce.
|
||||
(Omkar Vinit Joshi via vinodkv)
|
||||
|
||||
MAPREDUCE-4985. Add compression option to TestDFSIO usage.
|
||||
(Plamen Jeliazkov via shv)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -195,6 +224,109 @@ Release 2.0.4-beta - UNRELEASED
|
|||
MAPREDUCE-5008. Merger progress miscounts with respect to EOF_MARKER.
|
||||
(Sandy Ryza via tomwhite)
|
||||
|
||||
MAPREDUCE-4693. History server should include counters for failed tasks.
|
||||
(Xuan Gong via sseth)
|
||||
|
||||
MAPREDUCE-4896. mapred queue -info spits out ugly exception when queue does
|
||||
not exist. (sandyr via tucu)
|
||||
|
||||
MAPREDUCE-3685. Fix bugs in MergeManager to ensure compression codec is
|
||||
appropriately used and that on-disk segments are correctly sorted on
|
||||
file-size. (Anty Rao and Ravi Prakash via acmurthy)
|
||||
|
||||
MAPREDUCE-4571. TestHsWebServicesJobs fails on jdk7. (tgraves via tucu)
|
||||
|
||||
MAPREDUCE-4716. TestHsWebServicesJobsQuery.testJobsQueryStateInvalid
|
||||
fails with jdk7. (tgraves via tucu)
|
||||
|
||||
MAPREDUCE-5075. DistCp leaks input file handles since ThrottledInputStream
|
||||
does not close the wrapped InputStream. (Chris Nauroth via szetszwo)
|
||||
|
||||
MAPREDUCE-3872. Fix an event handling races in ContainerLauncherImpl.
|
||||
(Robert Kanter via sseth)
|
||||
|
||||
MAPREDUCE-5062. Fix MR AM to read max-retries from the RM. (Zhijie Shen via
|
||||
vinodkv)
|
||||
|
||||
MAPREDUCE-3829. [Gridmix] Gridmix should give better error message when
|
||||
input data directory already exists and -generate opton is
|
||||
given.(ravigummadi via tgraves)
|
||||
|
||||
MAPREDUCE-2722. [Gridmix] Gridmix simulated job's map's hdfsBytesRead
|
||||
counter is wrong when compressed input is used.(ravigummadi via tgraves)
|
||||
|
||||
MAPREDUCE-3953. [Gridmix] Gridmix throws NPE and does not simulate a
|
||||
job if the trace contains null taskStatus for a task. (ravigummadi via
|
||||
tgraves)
|
||||
|
||||
MAPREDUCE-4087. [Gridmix] GenerateDistCacheData job of Gridmix can
|
||||
become slow in some cases (ravigummadi via tgraves).
|
||||
|
||||
MAPREDUCE-5077. Remove mapreduce.util.ResourceCalculatorPlugin and related
|
||||
code. (Karthik Kambatla via sseth)
|
||||
|
||||
MAPREDUCE-4083. [Gridmix] NPE in cpu emulation. (amarrk via tgraves)
|
||||
|
||||
MAPREDUCE-4100. [Gridmix] Bug fixed in compression emulation feature for
|
||||
map only jobs. (amarrk via tgraves)
|
||||
|
||||
MAPREDUCE-4356. [Rumen] Provide access to the method
|
||||
ParsedTask.obtainTaskAttempts(). (ravigummadi via tgraves)
|
||||
|
||||
MAPREDUCE-4149. [Rumen] Rumen fails to parse certain counter
|
||||
strings. (ravigummadi via tgraves)
|
||||
|
||||
MAPREDUCE-3757. [Rumen] Fixed Rumen Folder to adjust shuffleFinished and
|
||||
sortFinished times when needed. (Ravi Gummadi via tgraves)
|
||||
|
||||
MAPREDUCE-5138. Fix LocalDistributedCacheManager after YARN-112. (Omkar Vinit
|
||||
Joshi via vinodkv)
|
||||
|
||||
MAPREDUCE-5113. Streaming input/output types are ignored with java
|
||||
mapper/reducer. (sandyr via tucu)
|
||||
|
||||
MAPREDUCE-5098. Fix findbugs warnings in gridmix. (kkambatl via tucu)
|
||||
|
||||
MAPREDUCE-5086. MR app master deletes staging dir when sent a reboot
|
||||
command from the RM. (Jian He via jlowe)
|
||||
|
||||
MAPREDUCE-5137. AM web UI: clicking on Map Task results in 500 error
|
||||
(Thomas Graves via jlowe)
|
||||
|
||||
MAPREDUCE-5136. TestJobImpl->testJobNoTasks fails with IBM JAVA (Amir
|
||||
Sanjar via jlowe)
|
||||
|
||||
MAPREDUCE-5139. Update MR AM to use the modified startContainer API after
|
||||
YARN-486. (Xuan Gong via vinodkv)
|
||||
|
||||
Release 2.0.4-alpha - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
||||
MAPREDUCE-5006. Fix failing streaming tests due to MAPREDUCE-4994.
|
||||
(Sandy Ryza via tomwhite)
|
||||
|
||||
MAPREDUCE-5088. MR Client gets an renewer token exception while Oozie is
|
||||
submitting a job (Daryn Sharp via cos)
|
||||
|
||||
MAPREDUCE-5117. Changed MRClientProtocolPBClientImpl to be closeable and thus
|
||||
fix failures in renewal of HistoryServer's delegations tokens. (Siddharth
|
||||
Seth via vinodkv)
|
||||
|
||||
MAPREDUCE-5083. MiniMRCluster should use a random component when creating an
|
||||
actual cluster (Siddharth Seth via hitesh)
|
||||
|
||||
MAPREDUCE-5094. Disabled memory monitoring by default in MiniMRYarnCluster
|
||||
to avoid some downstream tests failing. (Siddharth Seth via vinodkv)
|
||||
|
||||
Release 2.0.3-alpha - 2013-02-06
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -711,6 +843,18 @@ Release 2.0.0-alpha - 05-23-2012
|
|||
MAPREDUCE-4444. nodemanager fails to start when one of the local-dirs is
|
||||
bad (Jason Lowe via bobby)
|
||||
|
||||
Release 0.23.8 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
||||
Release 0.23.7 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -725,6 +869,12 @@ Release 0.23.7 - UNRELEASED
|
|||
MAPREDUCE-4989. JSONify DataTables input data for Attempts page (Ravi
|
||||
Prakash via jlowe)
|
||||
|
||||
MAPREDUCE-5027. Shuffle does not limit number of outstanding connections
|
||||
(Robert Parker via jeagles)
|
||||
|
||||
MAPREDUCE-4972. Coverage fixing for org.apache.hadoop.mapreduce.jobhistory
|
||||
(Aleksey Gorshkov via bobby)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
MAPREDUCE-4946. Fix a performance problem for large jobs by reducing the
|
||||
|
@ -744,6 +894,35 @@ Release 0.23.7 - UNRELEASED
|
|||
MAPREDUCE-5009. Killing the Task Attempt slated for commit does not clear
|
||||
the value from the Task commitAttempt member (Robert Parker via jeagles)
|
||||
|
||||
MAPREDUCE-4871. AM uses mapreduce.jobtracker.split.metainfo.maxsize but
|
||||
mapred-default has mapreduce.job.split.metainfo.maxsize (Jason Lowe via
|
||||
jeagles)
|
||||
|
||||
MAPREDUCE-4794. DefaultSpeculator generates error messages on normal
|
||||
shutdown (Jason Lowe via jeagles)
|
||||
|
||||
MAPREDUCE-5043. Fetch failure processing can cause AM event queue to
|
||||
backup and eventually OOM (Jason Lowe via bobby)
|
||||
|
||||
MAPREDUCE-5023. History Server Web Services missing Job Counters (Ravi
|
||||
Prakash via tgraves)
|
||||
|
||||
MAPREDUCE-5060. Fetch failures that time out only count against the first
|
||||
map task (Robert Joseph Evans via jlowe)
|
||||
|
||||
MAPREDUCE-5042. Reducer unable to fetch for a map task that was recovered
|
||||
(Jason Lowe via bobby)
|
||||
|
||||
MAPREDUCE-5053. java.lang.InternalError from decompression codec cause
|
||||
reducer to fail (Robert Parker via jeagles)
|
||||
|
||||
MAPREDUCE-4991. coverage for gridmix (Aleksey Gorshkov via tgraves)
|
||||
|
||||
MAPREDUCE-5007. fix coverage org.apache.hadoop.mapreduce.v2.hs (Aleksey
|
||||
Gorshkov via tgraves)
|
||||
|
||||
MAPREDUCE-5137. AM web UI: clicking on Map Task results in 500 error
|
||||
(Thomas Graves via jlowe)
|
||||
|
||||
Release 0.23.6 - UNRELEASED
|
||||
|
||||
|
|
|
@ -24,9 +24,12 @@ import java.lang.reflect.Constructor;
|
|||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -46,6 +49,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
|
|||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.AMStartedEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.EventReader;
|
||||
|
@ -54,6 +58,9 @@ import org.apache.hadoop.mapreduce.jobhistory.HistoryEvent;
|
|||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryCopyService;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
|
||||
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.security.TokenCache;
|
||||
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
|
||||
|
@ -61,6 +68,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
|
|||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
||||
import org.apache.hadoop.mapreduce.v2.app.client.MRClientService;
|
||||
|
@ -74,6 +82,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobFinishEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobStartEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
||||
|
@ -84,8 +93,6 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
|
|||
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl;
|
||||
import org.apache.hadoop.mapreduce.v2.app.local.LocalContainerAllocator;
|
||||
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
|
||||
import org.apache.hadoop.mapreduce.v2.app.recover.Recovery;
|
||||
import org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
|
||||
|
@ -94,6 +101,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
|
|||
import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
|
||||
import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
|
||||
import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
|
@ -167,7 +175,6 @@ public class MRAppMaster extends CompositeService {
|
|||
private AppContext context;
|
||||
private Dispatcher dispatcher;
|
||||
private ClientService clientService;
|
||||
private Recovery recoveryServ;
|
||||
private ContainerAllocator containerAllocator;
|
||||
private ContainerLauncher containerLauncher;
|
||||
private EventHandler<CommitterEvent> committerEventHandler;
|
||||
|
@ -180,7 +187,6 @@ public class MRAppMaster extends CompositeService {
|
|||
private OutputCommitter committer;
|
||||
private JobEventDispatcher jobEventDispatcher;
|
||||
private JobHistoryEventHandler jobHistoryEventHandler;
|
||||
private boolean inRecovery = false;
|
||||
private SpeculatorEventDispatcher speculatorEventDispatcher;
|
||||
|
||||
private Job job;
|
||||
|
@ -193,6 +199,8 @@ public class MRAppMaster extends CompositeService {
|
|||
private String shutDownMessage = null;
|
||||
JobStateInternal forcedState = null;
|
||||
|
||||
private long recoveredJobStartTime = 0;
|
||||
|
||||
public MRAppMaster(ApplicationAttemptId applicationAttemptId,
|
||||
ContainerId containerId, String nmHost, int nmPort, int nmHttpPort,
|
||||
long appSubmitTime, int maxAppAttempts) {
|
||||
|
@ -340,34 +348,9 @@ public class MRAppMaster extends CompositeService {
|
|||
}
|
||||
} else {
|
||||
committer = createOutputCommitter(conf);
|
||||
boolean recoveryEnabled = conf.getBoolean(
|
||||
MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
||||
boolean recoverySupportedByCommitter = committer.isRecoverySupported();
|
||||
|
||||
// If a shuffle secret was not provided by the job client then this app
|
||||
// attempt will generate one. However that disables recovery if there
|
||||
// are reducers as the shuffle secret would be app attempt specific.
|
||||
boolean shuffleKeyValidForRecovery = (numReduceTasks > 0 &&
|
||||
TokenCache.getShuffleSecretKey(fsTokens) != null);
|
||||
|
||||
if (recoveryEnabled && recoverySupportedByCommitter
|
||||
&& shuffleKeyValidForRecovery && appAttemptID.getAttemptId() > 1) {
|
||||
LOG.info("Recovery is enabled. "
|
||||
+ "Will try to recover from previous life on best effort basis.");
|
||||
recoveryServ = createRecoveryService(context);
|
||||
addIfService(recoveryServ);
|
||||
dispatcher = recoveryServ.getDispatcher();
|
||||
clock = recoveryServ.getClock();
|
||||
inRecovery = true;
|
||||
} else {
|
||||
LOG.info("Not starting RecoveryService: recoveryEnabled: "
|
||||
+ recoveryEnabled + " recoverySupportedByCommitter: "
|
||||
+ recoverySupportedByCommitter + " shuffleKeyValidForRecovery: "
|
||||
+ shuffleKeyValidForRecovery + " ApplicationAttemptID: "
|
||||
+ appAttemptID.getAttemptId());
|
||||
dispatcher = createDispatcher();
|
||||
addIfService(dispatcher);
|
||||
}
|
||||
|
||||
//service to handle requests from JobClient
|
||||
clientService = createClientService(context);
|
||||
|
@ -549,8 +532,14 @@ public class MRAppMaster extends CompositeService {
|
|||
}
|
||||
|
||||
try {
|
||||
//We are finishing cleanly so this is the last retry
|
||||
//if isLastAMRetry comes as true, should never set it to false
|
||||
if ( !isLastAMRetry){
|
||||
if (((JobImpl)job).getInternalState() != JobStateInternal.REBOOT) {
|
||||
LOG.info("We are finishing cleanly so this is the last retry");
|
||||
isLastAMRetry = true;
|
||||
}
|
||||
}
|
||||
notifyIsLastAMRetry(isLastAMRetry);
|
||||
// Stop all services
|
||||
// This will also send the final report to the ResourceManager
|
||||
LOG.info("Calling stop for all the services");
|
||||
|
@ -589,15 +578,6 @@ public class MRAppMaster extends CompositeService {
|
|||
return new JobFinishEventHandler();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the recovery service.
|
||||
* @return an instance of the recovery service.
|
||||
*/
|
||||
protected Recovery createRecoveryService(AppContext appContext) {
|
||||
return new RecoveryService(appContext.getApplicationAttemptId(),
|
||||
appContext.getClock(), getCommitter(), isNewApiCommitter());
|
||||
}
|
||||
|
||||
/** Create and initialize (but don't start) a single job.
|
||||
* @param forcedState a state to force the job into or null for normal operation.
|
||||
* @param diagnostic a diagnostic message to include with the job.
|
||||
|
@ -609,7 +589,8 @@ public class MRAppMaster extends CompositeService {
|
|||
Job newJob =
|
||||
new JobImpl(jobId, appAttemptID, conf, dispatcher.getEventHandler(),
|
||||
taskAttemptListener, jobTokenSecretManager, fsTokens, clock,
|
||||
completedTasksFromPreviousRun, metrics, newApiCommitter,
|
||||
completedTasksFromPreviousRun, metrics,
|
||||
committer, newApiCommitter,
|
||||
currentUser.getUserName(), appSubmitTime, amInfos, context,
|
||||
forcedState, diagnostic);
|
||||
((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
|
||||
|
@ -972,18 +953,8 @@ public class MRAppMaster extends CompositeService {
|
|||
public void start() {
|
||||
|
||||
amInfos = new LinkedList<AMInfo>();
|
||||
|
||||
// Pull completedTasks etc from recovery
|
||||
if (inRecovery) {
|
||||
completedTasksFromPreviousRun = recoveryServ.getCompletedTasks();
|
||||
amInfos = recoveryServ.getAMInfos();
|
||||
} else {
|
||||
// Get the amInfos anyways irrespective of whether recovery is enabled or
|
||||
// not IF this is not the first AM generation
|
||||
if (appAttemptID.getAttemptId() != 1) {
|
||||
amInfos.addAll(readJustAMInfos());
|
||||
}
|
||||
}
|
||||
completedTasksFromPreviousRun = new HashMap<TaskId, TaskInfo>();
|
||||
processRecovery();
|
||||
|
||||
// Current an AMInfo for the current AM generation.
|
||||
AMInfo amInfo =
|
||||
|
@ -1045,13 +1016,105 @@ public class MRAppMaster extends CompositeService {
|
|||
startJobs();
|
||||
}
|
||||
|
||||
private void processRecovery() {
|
||||
if (appAttemptID.getAttemptId() == 1) {
|
||||
return; // no need to recover on the first attempt
|
||||
}
|
||||
|
||||
boolean recoveryEnabled = getConfig().getBoolean(
|
||||
MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE,
|
||||
MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE_DEFAULT);
|
||||
boolean recoverySupportedByCommitter =
|
||||
committer != null && committer.isRecoverySupported();
|
||||
|
||||
// If a shuffle secret was not provided by the job client then this app
|
||||
// attempt will generate one. However that disables recovery if there
|
||||
// are reducers as the shuffle secret would be app attempt specific.
|
||||
int numReduceTasks = getConfig().getInt(MRJobConfig.NUM_REDUCES, 0);
|
||||
boolean shuffleKeyValidForRecovery = (numReduceTasks > 0 &&
|
||||
TokenCache.getShuffleSecretKey(fsTokens) != null);
|
||||
|
||||
if (recoveryEnabled && recoverySupportedByCommitter
|
||||
&& shuffleKeyValidForRecovery) {
|
||||
LOG.info("Recovery is enabled. "
|
||||
+ "Will try to recover from previous life on best effort basis.");
|
||||
try {
|
||||
parsePreviousJobHistory();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to parse prior job history, aborting recovery", e);
|
||||
// try to get just the AMInfos
|
||||
amInfos.addAll(readJustAMInfos());
|
||||
}
|
||||
} else {
|
||||
LOG.info("Will not try to recover. recoveryEnabled: "
|
||||
+ recoveryEnabled + " recoverySupportedByCommitter: "
|
||||
+ recoverySupportedByCommitter + " shuffleKeyValidForRecovery: "
|
||||
+ shuffleKeyValidForRecovery + " ApplicationAttemptID: "
|
||||
+ appAttemptID.getAttemptId());
|
||||
// Get the amInfos anyways whether recovery is enabled or not
|
||||
amInfos.addAll(readJustAMInfos());
|
||||
}
|
||||
}
|
||||
|
||||
private static FSDataInputStream getPreviousJobHistoryStream(
|
||||
Configuration conf, ApplicationAttemptId appAttemptId)
|
||||
throws IOException {
|
||||
Path historyFile = JobHistoryUtils.getPreviousJobHistoryPath(conf,
|
||||
appAttemptId);
|
||||
LOG.info("Previous history file is at " + historyFile);
|
||||
return historyFile.getFileSystem(conf).open(historyFile);
|
||||
}
|
||||
|
||||
private void parsePreviousJobHistory() throws IOException {
|
||||
FSDataInputStream in = getPreviousJobHistoryStream(getConfig(),
|
||||
appAttemptID);
|
||||
JobHistoryParser parser = new JobHistoryParser(in);
|
||||
JobInfo jobInfo = parser.parse();
|
||||
Exception parseException = parser.getParseException();
|
||||
if (parseException != null) {
|
||||
LOG.info("Got an error parsing job-history file" +
|
||||
", ignoring incomplete events.", parseException);
|
||||
}
|
||||
Map<org.apache.hadoop.mapreduce.TaskID, TaskInfo> taskInfos = jobInfo
|
||||
.getAllTasks();
|
||||
for (TaskInfo taskInfo : taskInfos.values()) {
|
||||
if (TaskState.SUCCEEDED.toString().equals(taskInfo.getTaskStatus())) {
|
||||
Iterator<Entry<TaskAttemptID, TaskAttemptInfo>> taskAttemptIterator =
|
||||
taskInfo.getAllTaskAttempts().entrySet().iterator();
|
||||
while (taskAttemptIterator.hasNext()) {
|
||||
Map.Entry<TaskAttemptID, TaskAttemptInfo> currentEntry = taskAttemptIterator.next();
|
||||
if (!jobInfo.getAllCompletedTaskAttempts().containsKey(currentEntry.getKey())) {
|
||||
taskAttemptIterator.remove();
|
||||
}
|
||||
}
|
||||
completedTasksFromPreviousRun
|
||||
.put(TypeConverter.toYarn(taskInfo.getTaskId()), taskInfo);
|
||||
LOG.info("Read from history task "
|
||||
+ TypeConverter.toYarn(taskInfo.getTaskId()));
|
||||
}
|
||||
}
|
||||
LOG.info("Read completed tasks from history "
|
||||
+ completedTasksFromPreviousRun.size());
|
||||
recoveredJobStartTime = jobInfo.getLaunchTime();
|
||||
|
||||
// recover AMInfos
|
||||
List<JobHistoryParser.AMInfo> jhAmInfoList = jobInfo.getAMInfos();
|
||||
if (jhAmInfoList != null) {
|
||||
for (JobHistoryParser.AMInfo jhAmInfo : jhAmInfoList) {
|
||||
AMInfo amInfo = MRBuilderUtils.newAMInfo(jhAmInfo.getAppAttemptId(),
|
||||
jhAmInfo.getStartTime(), jhAmInfo.getContainerId(),
|
||||
jhAmInfo.getNodeManagerHost(), jhAmInfo.getNodeManagerPort(),
|
||||
jhAmInfo.getNodeManagerHttpPort());
|
||||
amInfos.add(amInfo);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private List<AMInfo> readJustAMInfos() {
|
||||
List<AMInfo> amInfos = new ArrayList<AMInfo>();
|
||||
FSDataInputStream inputStream = null;
|
||||
try {
|
||||
inputStream =
|
||||
RecoveryService.getPreviousJobHistoryFileStream(getConfig(),
|
||||
appAttemptID);
|
||||
inputStream = getPreviousJobHistoryStream(getConfig(), appAttemptID);
|
||||
EventReader jobHistoryEventReader = new EventReader(inputStream);
|
||||
|
||||
// All AMInfos are contiguous. Track when the first AMStartedEvent
|
||||
|
@ -1102,7 +1165,8 @@ public class MRAppMaster extends CompositeService {
|
|||
@SuppressWarnings("unchecked")
|
||||
protected void startJobs() {
|
||||
/** create a job-start event to get this ball rolling */
|
||||
JobEvent startJobEvent = new JobEvent(job.getID(), JobEventType.JOB_START);
|
||||
JobEvent startJobEvent = new JobStartEvent(job.getID(),
|
||||
recoveredJobStartTime);
|
||||
/** send the job-start event. this triggers the job execution. */
|
||||
dispatcher.getEventHandler().handle(startJobEvent);
|
||||
}
|
||||
|
@ -1273,15 +1337,21 @@ public class MRAppMaster extends CompositeService {
|
|||
if(appMaster.containerAllocator instanceof ContainerAllocatorRouter) {
|
||||
((ContainerAllocatorRouter) appMaster.containerAllocator)
|
||||
.setSignalled(true);
|
||||
((ContainerAllocatorRouter) appMaster.containerAllocator)
|
||||
.setShouldUnregister(appMaster.isLastAMRetry);
|
||||
}
|
||||
appMaster.notifyIsLastAMRetry(appMaster.isLastAMRetry);
|
||||
appMaster.stop();
|
||||
}
|
||||
}
|
||||
|
||||
if(appMaster.jobHistoryEventHandler != null) {
|
||||
appMaster.jobHistoryEventHandler
|
||||
.setForcejobCompletion(appMaster.isLastAMRetry);
|
||||
public void notifyIsLastAMRetry(boolean isLastAMRetry){
|
||||
if(containerAllocator instanceof ContainerAllocatorRouter) {
|
||||
LOG.info("Notify RMCommunicator isAMLastRetry: " + isLastAMRetry);
|
||||
((ContainerAllocatorRouter) containerAllocator)
|
||||
.setShouldUnregister(isLastAMRetry);
|
||||
}
|
||||
appMaster.stop();
|
||||
if(jobHistoryEventHandler != null) {
|
||||
LOG.info("Notify JHEH isAMLastRetry: " + isLastAMRetry);
|
||||
jobHistoryEventHandler.setForcejobCompletion(isLastAMRetry);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -30,5 +30,6 @@ public enum JobStateInternal {
|
|||
KILL_WAIT,
|
||||
KILL_ABORT,
|
||||
KILLED,
|
||||
ERROR
|
||||
ERROR,
|
||||
REBOOT
|
||||
}
|
||||
|
|
|
@ -54,6 +54,6 @@ public enum JobEventType {
|
|||
JOB_TASK_ATTEMPT_FETCH_FAILURE,
|
||||
|
||||
//Producer:RMContainerAllocator
|
||||
JOB_UPDATED_NODES
|
||||
|
||||
JOB_UPDATED_NODES,
|
||||
JOB_AM_REBOOT
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/*
|
||||
/**
|
||||
* 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
|
||||
|
@ -15,6 +15,25 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
package org.apache.hadoop.mapreduce.v2.app.recover;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app.job.event;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
|
||||
public class JobStartEvent extends JobEvent {
|
||||
|
||||
long recoveredJobStartTime;
|
||||
|
||||
public JobStartEvent(JobId jobID) {
|
||||
this(jobID, 0);
|
||||
}
|
||||
|
||||
public JobStartEvent(JobId jobID, long recoveredJobStartTime) {
|
||||
super(jobID, JobEventType.JOB_START);
|
||||
this.recoveredJobStartTime = recoveredJobStartTime;
|
||||
}
|
||||
|
||||
public long getRecoveredJobStartTime() {
|
||||
return recoveredJobStartTime;
|
||||
}
|
||||
}
|
|
@ -26,6 +26,7 @@ public enum TaskAttemptEventType {
|
|||
//Producer:Task
|
||||
TA_SCHEDULE,
|
||||
TA_RESCHEDULE,
|
||||
TA_RECOVER,
|
||||
|
||||
//Producer:Client, Task
|
||||
TA_KILL,
|
||||
|
|
|
@ -0,0 +1,50 @@
|
|||
/**
|
||||
* 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.mapreduce.v2.app.job.event;
|
||||
|
||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
||||
|
||||
public class TaskAttemptRecoverEvent extends TaskAttemptEvent {
|
||||
|
||||
private TaskAttemptInfo taInfo;
|
||||
private OutputCommitter committer;
|
||||
private boolean recoverAttemptOutput;
|
||||
|
||||
public TaskAttemptRecoverEvent(TaskAttemptId id, TaskAttemptInfo taInfo,
|
||||
OutputCommitter committer, boolean recoverOutput) {
|
||||
super(id, TaskAttemptEventType.TA_RECOVER);
|
||||
this.taInfo = taInfo;
|
||||
this.committer = committer;
|
||||
this.recoverAttemptOutput = recoverOutput;
|
||||
}
|
||||
|
||||
public TaskAttemptInfo getTaskAttemptInfo() {
|
||||
return taInfo;
|
||||
}
|
||||
|
||||
public OutputCommitter getCommitter() {
|
||||
return committer;
|
||||
}
|
||||
|
||||
public boolean getRecoverOutput() {
|
||||
return recoverAttemptOutput;
|
||||
}
|
||||
}
|
|
@ -28,6 +28,7 @@ public enum TaskEventType {
|
|||
|
||||
//Producer:Job
|
||||
T_SCHEDULE,
|
||||
T_RECOVER,
|
||||
|
||||
//Producer:Speculator
|
||||
T_ADD_SPEC_ATTEMPT,
|
||||
|
|
|
@ -0,0 +1,50 @@
|
|||
/**
|
||||
* 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.mapreduce.v2.app.job.event;
|
||||
|
||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
|
||||
public class TaskRecoverEvent extends TaskEvent {
|
||||
|
||||
private TaskInfo taskInfo;
|
||||
private OutputCommitter committer;
|
||||
private boolean recoverTaskOutput;
|
||||
|
||||
public TaskRecoverEvent(TaskId taskID, TaskInfo taskInfo,
|
||||
OutputCommitter committer, boolean recoverTaskOutput) {
|
||||
super(taskID, TaskEventType.T_RECOVER);
|
||||
this.taskInfo = taskInfo;
|
||||
this.committer = committer;
|
||||
this.recoverTaskOutput = recoverTaskOutput;
|
||||
}
|
||||
|
||||
public TaskInfo getTaskInfo() {
|
||||
return taskInfo;
|
||||
}
|
||||
|
||||
public OutputCommitter getOutputCommitter() {
|
||||
return committer;
|
||||
}
|
||||
|
||||
public boolean getRecoverTaskOutput() {
|
||||
return recoverTaskOutput;
|
||||
}
|
||||
}
|
|
@ -49,6 +49,7 @@ import org.apache.hadoop.mapreduce.Counters;
|
|||
import org.apache.hadoop.mapreduce.JobACL;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobFinishedEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
||||
|
@ -92,6 +93,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobFinishEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobSetupFailedEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobStartEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptCompletedEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptFetchFailureEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
|
||||
|
@ -101,6 +103,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskRecoverEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
||||
|
@ -159,6 +162,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
private final Lock writeLock;
|
||||
private final JobId jobId;
|
||||
private final String jobName;
|
||||
private final OutputCommitter committer;
|
||||
private final boolean newApiCommitter;
|
||||
private final org.apache.hadoop.mapreduce.JobID oldJobId;
|
||||
private final TaskAttemptListener taskAttemptListener;
|
||||
|
@ -215,6 +219,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
|
||||
private static final InternalErrorTransition
|
||||
INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
|
||||
private static final InternalRebootTransition
|
||||
INTERNAL_REBOOT_TRANSITION = new InternalRebootTransition();
|
||||
private static final TaskAttemptCompletedEventTransition
|
||||
TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION =
|
||||
new TaskAttemptCompletedEventTransition();
|
||||
|
@ -246,6 +252,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
.addTransition(JobStateInternal.NEW, JobStateInternal.ERROR,
|
||||
JobEventType.INTERNAL_ERROR,
|
||||
INTERNAL_ERROR_TRANSITION)
|
||||
.addTransition(JobStateInternal.NEW, JobStateInternal.REBOOT,
|
||||
JobEventType.JOB_AM_REBOOT,
|
||||
INTERNAL_REBOOT_TRANSITION)
|
||||
// Ignore-able events
|
||||
.addTransition(JobStateInternal.NEW, JobStateInternal.NEW,
|
||||
JobEventType.JOB_UPDATED_NODES)
|
||||
|
@ -265,6 +274,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
.addTransition(JobStateInternal.INITED, JobStateInternal.ERROR,
|
||||
JobEventType.INTERNAL_ERROR,
|
||||
INTERNAL_ERROR_TRANSITION)
|
||||
.addTransition(JobStateInternal.INITED, JobStateInternal.REBOOT,
|
||||
JobEventType.JOB_AM_REBOOT,
|
||||
INTERNAL_REBOOT_TRANSITION)
|
||||
// Ignore-able events
|
||||
.addTransition(JobStateInternal.INITED, JobStateInternal.INITED,
|
||||
JobEventType.JOB_UPDATED_NODES)
|
||||
|
@ -287,6 +299,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
.addTransition(JobStateInternal.SETUP, JobStateInternal.ERROR,
|
||||
JobEventType.INTERNAL_ERROR,
|
||||
INTERNAL_ERROR_TRANSITION)
|
||||
.addTransition(JobStateInternal.SETUP, JobStateInternal.REBOOT,
|
||||
JobEventType.JOB_AM_REBOOT,
|
||||
INTERNAL_REBOOT_TRANSITION)
|
||||
// Ignore-able events
|
||||
.addTransition(JobStateInternal.SETUP, JobStateInternal.SETUP,
|
||||
JobEventType.JOB_UPDATED_NODES)
|
||||
|
@ -327,6 +342,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
JobStateInternal.RUNNING,
|
||||
JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
|
||||
INTERNAL_ERROR_TRANSITION)
|
||||
.addTransition(JobStateInternal.RUNNING, JobStateInternal.REBOOT,
|
||||
JobEventType.JOB_AM_REBOOT,
|
||||
INTERNAL_REBOOT_TRANSITION)
|
||||
|
||||
// Transitions from KILL_WAIT state.
|
||||
.addTransition
|
||||
|
@ -352,7 +370,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
EnumSet.of(JobEventType.JOB_KILL,
|
||||
JobEventType.JOB_UPDATED_NODES,
|
||||
JobEventType.JOB_MAP_TASK_RESCHEDULED,
|
||||
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE))
|
||||
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
|
||||
JobEventType.JOB_AM_REBOOT))
|
||||
|
||||
// Transitions from COMMITTING state
|
||||
.addTransition(JobStateInternal.COMMITTING,
|
||||
|
@ -377,6 +396,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
.addTransition(JobStateInternal.COMMITTING,
|
||||
JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
|
||||
INTERNAL_ERROR_TRANSITION)
|
||||
.addTransition(JobStateInternal.COMMITTING, JobStateInternal.REBOOT,
|
||||
JobEventType.JOB_AM_REBOOT,
|
||||
INTERNAL_REBOOT_TRANSITION)
|
||||
// Ignore-able events
|
||||
.addTransition(JobStateInternal.COMMITTING,
|
||||
JobStateInternal.COMMITTING,
|
||||
|
@ -397,7 +419,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
.addTransition(JobStateInternal.SUCCEEDED, JobStateInternal.SUCCEEDED,
|
||||
EnumSet.of(JobEventType.JOB_KILL,
|
||||
JobEventType.JOB_UPDATED_NODES,
|
||||
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE))
|
||||
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
|
||||
JobEventType.JOB_AM_REBOOT))
|
||||
|
||||
// Transitions from FAIL_ABORT state
|
||||
.addTransition(JobStateInternal.FAIL_ABORT,
|
||||
|
@ -425,7 +448,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
JobEventType.JOB_MAP_TASK_RESCHEDULED,
|
||||
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
|
||||
JobEventType.JOB_COMMIT_COMPLETED,
|
||||
JobEventType.JOB_COMMIT_FAILED))
|
||||
JobEventType.JOB_COMMIT_FAILED,
|
||||
JobEventType.JOB_AM_REBOOT))
|
||||
|
||||
// Transitions from KILL_ABORT state
|
||||
.addTransition(JobStateInternal.KILL_ABORT,
|
||||
|
@ -452,7 +476,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
JobEventType.JOB_SETUP_COMPLETED,
|
||||
JobEventType.JOB_SETUP_FAILED,
|
||||
JobEventType.JOB_COMMIT_COMPLETED,
|
||||
JobEventType.JOB_COMMIT_FAILED))
|
||||
JobEventType.JOB_COMMIT_FAILED,
|
||||
JobEventType.JOB_AM_REBOOT))
|
||||
|
||||
// Transitions from FAILED state
|
||||
.addTransition(JobStateInternal.FAILED, JobStateInternal.FAILED,
|
||||
|
@ -476,7 +501,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
JobEventType.JOB_SETUP_FAILED,
|
||||
JobEventType.JOB_COMMIT_COMPLETED,
|
||||
JobEventType.JOB_COMMIT_FAILED,
|
||||
JobEventType.JOB_ABORT_COMPLETED))
|
||||
JobEventType.JOB_ABORT_COMPLETED,
|
||||
JobEventType.JOB_AM_REBOOT))
|
||||
|
||||
// Transitions from KILLED state
|
||||
.addTransition(JobStateInternal.KILLED, JobStateInternal.KILLED,
|
||||
|
@ -498,7 +524,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
JobEventType.JOB_SETUP_FAILED,
|
||||
JobEventType.JOB_COMMIT_COMPLETED,
|
||||
JobEventType.JOB_COMMIT_FAILED,
|
||||
JobEventType.JOB_ABORT_COMPLETED))
|
||||
JobEventType.JOB_ABORT_COMPLETED,
|
||||
JobEventType.JOB_AM_REBOOT))
|
||||
|
||||
// No transitions from INTERNAL_ERROR state. Ignore all.
|
||||
.addTransition(
|
||||
|
@ -517,9 +544,33 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
JobEventType.JOB_COMMIT_COMPLETED,
|
||||
JobEventType.JOB_COMMIT_FAILED,
|
||||
JobEventType.JOB_ABORT_COMPLETED,
|
||||
JobEventType.INTERNAL_ERROR))
|
||||
JobEventType.INTERNAL_ERROR,
|
||||
JobEventType.JOB_AM_REBOOT))
|
||||
.addTransition(JobStateInternal.ERROR, JobStateInternal.ERROR,
|
||||
JobEventType.JOB_COUNTER_UPDATE, COUNTER_UPDATE_TRANSITION)
|
||||
|
||||
// No transitions from AM_REBOOT state. Ignore all.
|
||||
.addTransition(
|
||||
JobStateInternal.REBOOT,
|
||||
JobStateInternal.REBOOT,
|
||||
EnumSet.of(JobEventType.JOB_INIT,
|
||||
JobEventType.JOB_KILL,
|
||||
JobEventType.JOB_TASK_COMPLETED,
|
||||
JobEventType.JOB_TASK_ATTEMPT_COMPLETED,
|
||||
JobEventType.JOB_MAP_TASK_RESCHEDULED,
|
||||
JobEventType.JOB_DIAGNOSTIC_UPDATE,
|
||||
JobEventType.JOB_UPDATED_NODES,
|
||||
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
|
||||
JobEventType.JOB_SETUP_COMPLETED,
|
||||
JobEventType.JOB_SETUP_FAILED,
|
||||
JobEventType.JOB_COMMIT_COMPLETED,
|
||||
JobEventType.JOB_COMMIT_FAILED,
|
||||
JobEventType.JOB_ABORT_COMPLETED,
|
||||
JobEventType.INTERNAL_ERROR,
|
||||
JobEventType.JOB_AM_REBOOT))
|
||||
.addTransition(JobStateInternal.REBOOT, JobStateInternal.REBOOT,
|
||||
JobEventType.JOB_COUNTER_UPDATE, COUNTER_UPDATE_TRANSITION)
|
||||
|
||||
// create the topology tables
|
||||
.installTopology();
|
||||
|
||||
|
@ -555,7 +606,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
JobTokenSecretManager jobTokenSecretManager,
|
||||
Credentials fsTokenCredentials, Clock clock,
|
||||
Map<TaskId, TaskInfo> completedTasksFromPreviousRun, MRAppMetrics metrics,
|
||||
boolean newApiCommitter, String userName,
|
||||
OutputCommitter committer, boolean newApiCommitter, String userName,
|
||||
long appSubmitTime, List<AMInfo> amInfos, AppContext appContext,
|
||||
JobStateInternal forcedState, String forcedDiagnostic) {
|
||||
this.applicationAttemptId = applicationAttemptId;
|
||||
|
@ -571,6 +622,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
this.queueName = conf.get(MRJobConfig.QUEUE_NAME, "default");
|
||||
this.appSubmitTime = appSubmitTime;
|
||||
this.oldJobId = TypeConverter.fromYarn(jobId);
|
||||
this.committer = committer;
|
||||
this.newApiCommitter = newApiCommitter;
|
||||
|
||||
this.taskAttemptListener = taskAttemptListener;
|
||||
|
@ -841,10 +893,16 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
}
|
||||
}
|
||||
|
||||
protected void scheduleTasks(Set<TaskId> taskIDs) {
|
||||
protected void scheduleTasks(Set<TaskId> taskIDs,
|
||||
boolean recoverTaskOutput) {
|
||||
for (TaskId taskID : taskIDs) {
|
||||
eventHandler.handle(new TaskEvent(taskID,
|
||||
TaskEventType.T_SCHEDULE));
|
||||
TaskInfo taskInfo = completedTasksFromPreviousRun.remove(taskID);
|
||||
if (taskInfo != null) {
|
||||
eventHandler.handle(new TaskRecoverEvent(taskID, taskInfo,
|
||||
committer, recoverTaskOutput));
|
||||
} else {
|
||||
eventHandler.handle(new TaskEvent(taskID, TaskEventType.T_SCHEDULE));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -904,6 +962,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
return JobState.RUNNING;
|
||||
case FAIL_ABORT:
|
||||
return JobState.FAILED;
|
||||
case REBOOT:
|
||||
return JobState.ERROR;
|
||||
default:
|
||||
return JobState.valueOf(smState.name());
|
||||
}
|
||||
|
@ -972,6 +1032,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
case KILLED:
|
||||
metrics.killedJob(this);
|
||||
break;
|
||||
case REBOOT:
|
||||
case ERROR:
|
||||
case FAILED:
|
||||
metrics.failedJob(this);
|
||||
|
@ -1255,7 +1316,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
job.conf.get(MRJobConfig.WORKFLOW_ID, ""),
|
||||
job.conf.get(MRJobConfig.WORKFLOW_NAME, ""),
|
||||
job.conf.get(MRJobConfig.WORKFLOW_NODE_NAME, ""),
|
||||
getWorkflowAdjacencies(job.conf));
|
||||
getWorkflowAdjacencies(job.conf),
|
||||
job.conf.get(MRJobConfig.WORKFLOW_TAGS, ""));
|
||||
job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse));
|
||||
//TODO JH Verify jobACLs, UserName via UGI?
|
||||
|
||||
|
@ -1370,7 +1432,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
job.conf, splits[i],
|
||||
job.taskAttemptListener,
|
||||
job.jobToken, job.fsTokens,
|
||||
job.clock, job.completedTasksFromPreviousRun,
|
||||
job.clock,
|
||||
job.applicationAttemptId.getAttemptId(),
|
||||
job.metrics, job.appContext);
|
||||
job.addTask(task);
|
||||
|
@ -1388,7 +1450,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
job.conf, job.numMapTasks,
|
||||
job.taskAttemptListener, job.jobToken,
|
||||
job.fsTokens, job.clock,
|
||||
job.completedTasksFromPreviousRun,
|
||||
job.applicationAttemptId.getAttemptId(),
|
||||
job.metrics, job.appContext);
|
||||
job.addTask(task);
|
||||
|
@ -1424,8 +1485,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
@Override
|
||||
public void transition(JobImpl job, JobEvent event) {
|
||||
job.setupProgress = 1.0f;
|
||||
job.scheduleTasks(job.mapTasks); // schedule (i.e., start) the maps
|
||||
job.scheduleTasks(job.reduceTasks);
|
||||
job.scheduleTasks(job.mapTasks, job.numReduceTasks == 0);
|
||||
job.scheduleTasks(job.reduceTasks, true);
|
||||
|
||||
// If we have no tasks, just transition to job completed
|
||||
if (job.numReduceTasks == 0 && job.numMapTasks == 0) {
|
||||
|
@ -1456,7 +1517,12 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
*/
|
||||
@Override
|
||||
public void transition(JobImpl job, JobEvent event) {
|
||||
JobStartEvent jse = (JobStartEvent) event;
|
||||
if (jse.getRecoveredJobStartTime() != 0) {
|
||||
job.startTime = jse.getRecoveredJobStartTime();
|
||||
} else {
|
||||
job.startTime = job.clock.getTime();
|
||||
}
|
||||
JobInitedEvent jie =
|
||||
new JobInitedEvent(job.oldJobId,
|
||||
job.startTime,
|
||||
|
@ -1898,8 +1964,17 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
}
|
||||
}
|
||||
|
||||
private static class InternalErrorTransition implements
|
||||
private static class InternalTerminationTransition implements
|
||||
SingleArcTransition<JobImpl, JobEvent> {
|
||||
JobStateInternal terminationState = null;
|
||||
String jobHistoryString = null;
|
||||
public InternalTerminationTransition(JobStateInternal stateInternal,
|
||||
String jobHistoryString) {
|
||||
this.terminationState = stateInternal;
|
||||
//mostly a hack for jbhistoryserver
|
||||
this.jobHistoryString = jobHistoryString;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void transition(JobImpl job, JobEvent event) {
|
||||
//TODO Is this JH event required.
|
||||
|
@ -1907,9 +1982,21 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
JobUnsuccessfulCompletionEvent failedEvent =
|
||||
new JobUnsuccessfulCompletionEvent(job.oldJobId,
|
||||
job.finishTime, 0, 0,
|
||||
JobStateInternal.ERROR.toString());
|
||||
jobHistoryString);
|
||||
job.eventHandler.handle(new JobHistoryEvent(job.jobId, failedEvent));
|
||||
job.finished(JobStateInternal.ERROR);
|
||||
job.finished(terminationState);
|
||||
}
|
||||
}
|
||||
|
||||
private static class InternalErrorTransition extends InternalTerminationTransition {
|
||||
public InternalErrorTransition(){
|
||||
super(JobStateInternal.ERROR, JobStateInternal.ERROR.toString());
|
||||
}
|
||||
}
|
||||
|
||||
private static class InternalRebootTransition extends InternalTerminationTransition {
|
||||
public InternalRebootTransition(){
|
||||
super(JobStateInternal.REBOOT, JobStateInternal.ERROR.toString());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,17 +18,13 @@
|
|||
|
||||
package org.apache.hadoop.mapreduce.v2.app.job.impl;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.MapTaskAttemptImpl;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
||||
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
|
||||
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
|
||||
|
@ -49,11 +45,10 @@ public class MapTaskImpl extends TaskImpl {
|
|||
TaskAttemptListener taskAttemptListener,
|
||||
Token<JobTokenIdentifier> jobToken,
|
||||
Credentials credentials, Clock clock,
|
||||
Map<TaskId, TaskInfo> completedTasksFromPreviousRun, int startCount,
|
||||
MRAppMetrics metrics, AppContext appContext) {
|
||||
int appAttemptId, MRAppMetrics metrics, AppContext appContext) {
|
||||
super(jobId, TaskType.MAP, partition, eventHandler, remoteJobConfFile,
|
||||
conf, taskAttemptListener, jobToken, credentials, clock,
|
||||
completedTasksFromPreviousRun, startCount, metrics, appContext);
|
||||
appAttemptId, metrics, appContext);
|
||||
this.taskSplitMetaInfo = taskSplitMetaInfo;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,16 +18,12 @@
|
|||
|
||||
package org.apache.hadoop.mapreduce.v2.app.job.impl;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.ReduceTaskAttemptImpl;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
||||
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
|
||||
|
@ -47,11 +43,10 @@ public class ReduceTaskImpl extends TaskImpl {
|
|||
int numMapTasks, TaskAttemptListener taskAttemptListener,
|
||||
Token<JobTokenIdentifier> jobToken,
|
||||
Credentials credentials, Clock clock,
|
||||
Map<TaskId, TaskInfo> completedTasksFromPreviousRun, int startCount,
|
||||
MRAppMetrics metrics, AppContext appContext) {
|
||||
int appAttemptId, MRAppMetrics metrics, AppContext appContext) {
|
||||
super(jobId, TaskType.REDUCE, partition, eventHandler, jobFile, conf,
|
||||
taskAttemptListener, jobToken, credentials, clock,
|
||||
completedTasksFromPreviousRun, startCount, metrics, appContext);
|
||||
appAttemptId, metrics, appContext);
|
||||
this.numMapTasks = numMapTasks;
|
||||
}
|
||||
|
||||
|
|
|
@ -56,10 +56,12 @@ import org.apache.hadoop.mapreduce.Counter;
|
|||
import org.apache.hadoop.mapreduce.Counters;
|
||||
import org.apache.hadoop.mapreduce.JobCounter;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hadoop.mapreduce.TaskCounter;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinishedEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.ReduceAttemptFinishedEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStartedEvent;
|
||||
|
@ -89,6 +91,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdate
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptRecoverEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
||||
|
@ -111,6 +114,7 @@ import org.apache.hadoop.yarn.Clock;
|
|||
import org.apache.hadoop.yarn.YarnException;
|
||||
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerToken;
|
||||
|
@ -204,6 +208,11 @@ public abstract class TaskAttemptImpl implements
|
|||
TaskAttemptEventType.TA_KILL, new KilledTransition())
|
||||
.addTransition(TaskAttemptStateInternal.NEW, TaskAttemptStateInternal.FAILED,
|
||||
TaskAttemptEventType.TA_FAILMSG, new FailedTransition())
|
||||
.addTransition(TaskAttemptStateInternal.NEW,
|
||||
EnumSet.of(TaskAttemptStateInternal.FAILED,
|
||||
TaskAttemptStateInternal.KILLED,
|
||||
TaskAttemptStateInternal.SUCCEEDED),
|
||||
TaskAttemptEventType.TA_RECOVER, new RecoverTransition())
|
||||
.addTransition(TaskAttemptStateInternal.NEW,
|
||||
TaskAttemptStateInternal.NEW,
|
||||
TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
|
||||
|
@ -759,8 +768,8 @@ public abstract class TaskAttemptImpl implements
|
|||
// The null fields are per-container and will be constructed for each
|
||||
// container separately.
|
||||
ContainerLaunchContext container = BuilderUtils
|
||||
.newContainerLaunchContext(null, conf
|
||||
.get(MRJobConfig.USER_NAME), null, localResources,
|
||||
.newContainerLaunchContext(conf
|
||||
.get(MRJobConfig.USER_NAME), localResources,
|
||||
environment, null, serviceData, taskCredentialsBuffer,
|
||||
applicationACLs);
|
||||
|
||||
|
@ -769,10 +778,9 @@ public abstract class TaskAttemptImpl implements
|
|||
|
||||
static ContainerLaunchContext createContainerLaunchContext(
|
||||
Map<ApplicationAccessType, String> applicationACLs,
|
||||
ContainerId containerID, Configuration conf,
|
||||
Token<JobTokenIdentifier> jobToken, Task remoteTask,
|
||||
Configuration conf, Token<JobTokenIdentifier> jobToken, Task remoteTask,
|
||||
final org.apache.hadoop.mapred.JobID oldJobId,
|
||||
Resource assignedCapability, WrappedJvmID jvmID,
|
||||
WrappedJvmID jvmID,
|
||||
TaskAttemptListener taskAttemptListener,
|
||||
Credentials credentials) {
|
||||
|
||||
|
@ -805,7 +813,7 @@ public abstract class TaskAttemptImpl implements
|
|||
|
||||
// Construct the actual Container
|
||||
ContainerLaunchContext container = BuilderUtils.newContainerLaunchContext(
|
||||
containerID, commonContainerSpec.getUser(), assignedCapability,
|
||||
commonContainerSpec.getUser(),
|
||||
commonContainerSpec.getLocalResources(), myEnv, commands,
|
||||
myServiceData, commonContainerSpec.getContainerTokens().duplicate(),
|
||||
applicationACLs);
|
||||
|
@ -1082,6 +1090,102 @@ public abstract class TaskAttemptImpl implements
|
|||
this.avataar = avataar;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public TaskAttemptStateInternal recover(TaskAttemptInfo taInfo,
|
||||
OutputCommitter committer, boolean recoverOutput) {
|
||||
containerID = taInfo.getContainerId();
|
||||
containerNodeId = ConverterUtils.toNodeId(taInfo.getHostname() + ":"
|
||||
+ taInfo.getPort());
|
||||
containerMgrAddress = StringInterner.weakIntern(
|
||||
containerNodeId.toString());
|
||||
nodeHttpAddress = StringInterner.weakIntern(taInfo.getHostname() + ":"
|
||||
+ taInfo.getHttpPort());
|
||||
computeRackAndLocality();
|
||||
launchTime = taInfo.getStartTime();
|
||||
finishTime = (taInfo.getFinishTime() != -1) ?
|
||||
taInfo.getFinishTime() : clock.getTime();
|
||||
shufflePort = taInfo.getShufflePort();
|
||||
trackerName = taInfo.getHostname();
|
||||
httpPort = taInfo.getHttpPort();
|
||||
sendLaunchedEvents();
|
||||
|
||||
reportedStatus.id = attemptId;
|
||||
reportedStatus.progress = 1.0f;
|
||||
reportedStatus.counters = taInfo.getCounters();
|
||||
reportedStatus.stateString = taInfo.getState();
|
||||
reportedStatus.phase = Phase.CLEANUP;
|
||||
reportedStatus.mapFinishTime = taInfo.getMapFinishTime();
|
||||
reportedStatus.shuffleFinishTime = taInfo.getShuffleFinishTime();
|
||||
reportedStatus.sortFinishTime = taInfo.getSortFinishTime();
|
||||
addDiagnosticInfo(taInfo.getError());
|
||||
|
||||
boolean needToClean = false;
|
||||
String recoveredState = taInfo.getTaskStatus();
|
||||
if (recoverOutput
|
||||
&& TaskAttemptState.SUCCEEDED.toString().equals(recoveredState)) {
|
||||
TaskAttemptContext tac = new TaskAttemptContextImpl(conf,
|
||||
TypeConverter.fromYarn(attemptId));
|
||||
try {
|
||||
committer.recoverTask(tac);
|
||||
LOG.info("Recovered output from task attempt " + attemptId);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Unable to recover task attempt " + attemptId, e);
|
||||
LOG.info("Task attempt " + attemptId + " will be recovered as KILLED");
|
||||
recoveredState = TaskAttemptState.KILLED.toString();
|
||||
needToClean = true;
|
||||
}
|
||||
}
|
||||
|
||||
TaskAttemptStateInternal attemptState;
|
||||
if (TaskAttemptState.SUCCEEDED.toString().equals(recoveredState)) {
|
||||
attemptState = TaskAttemptStateInternal.SUCCEEDED;
|
||||
reportedStatus.taskState = TaskAttemptState.SUCCEEDED;
|
||||
eventHandler.handle(createJobCounterUpdateEventTASucceeded(this));
|
||||
logAttemptFinishedEvent(attemptState);
|
||||
} else if (TaskAttemptState.FAILED.toString().equals(recoveredState)) {
|
||||
attemptState = TaskAttemptStateInternal.FAILED;
|
||||
reportedStatus.taskState = TaskAttemptState.FAILED;
|
||||
eventHandler.handle(createJobCounterUpdateEventTAFailed(this, false));
|
||||
TaskAttemptUnsuccessfulCompletionEvent tauce =
|
||||
createTaskAttemptUnsuccessfulCompletionEvent(this,
|
||||
TaskAttemptStateInternal.FAILED);
|
||||
eventHandler.handle(
|
||||
new JobHistoryEvent(attemptId.getTaskId().getJobId(), tauce));
|
||||
} else {
|
||||
if (!TaskAttemptState.KILLED.toString().equals(recoveredState)) {
|
||||
if (String.valueOf(recoveredState).isEmpty()) {
|
||||
LOG.info("TaskAttempt" + attemptId
|
||||
+ " had not completed, recovering as KILLED");
|
||||
} else {
|
||||
LOG.warn("TaskAttempt " + attemptId + " found in unexpected state "
|
||||
+ recoveredState + ", recovering as KILLED");
|
||||
}
|
||||
addDiagnosticInfo("Killed during application recovery");
|
||||
needToClean = true;
|
||||
}
|
||||
attemptState = TaskAttemptStateInternal.KILLED;
|
||||
reportedStatus.taskState = TaskAttemptState.KILLED;
|
||||
eventHandler.handle(createJobCounterUpdateEventTAKilled(this, false));
|
||||
TaskAttemptUnsuccessfulCompletionEvent tauce =
|
||||
createTaskAttemptUnsuccessfulCompletionEvent(this,
|
||||
TaskAttemptStateInternal.KILLED);
|
||||
eventHandler.handle(
|
||||
new JobHistoryEvent(attemptId.getTaskId().getJobId(), tauce));
|
||||
}
|
||||
|
||||
if (needToClean) {
|
||||
TaskAttemptContext tac = new TaskAttemptContextImpl(conf,
|
||||
TypeConverter.fromYarn(attemptId));
|
||||
try {
|
||||
committer.abortTask(tac);
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Task cleanup failed for attempt " + attemptId, e);
|
||||
}
|
||||
}
|
||||
|
||||
return attemptState;
|
||||
}
|
||||
|
||||
private static TaskAttemptState getExternalState(
|
||||
TaskAttemptStateInternal smState) {
|
||||
switch (smState) {
|
||||
|
@ -1122,6 +1226,24 @@ public abstract class TaskAttemptImpl implements
|
|||
}
|
||||
}
|
||||
|
||||
private void computeRackAndLocality() {
|
||||
nodeRackName = RackResolver.resolve(
|
||||
containerNodeId.getHost()).getNetworkLocation();
|
||||
|
||||
locality = Locality.OFF_SWITCH;
|
||||
if (dataLocalHosts.size() > 0) {
|
||||
String cHost = resolveHost(containerNodeId.getHost());
|
||||
if (dataLocalHosts.contains(cHost)) {
|
||||
locality = Locality.NODE_LOCAL;
|
||||
}
|
||||
}
|
||||
if (locality == Locality.OFF_SWITCH) {
|
||||
if (dataLocalRacks.contains(nodeRackName)) {
|
||||
locality = Locality.RACK_LOCAL;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static long computeSlotMillis(TaskAttemptImpl taskAttempt) {
|
||||
TaskType taskType = taskAttempt.getID().getTaskId().getTaskType();
|
||||
int slotMemoryReq =
|
||||
|
@ -1141,6 +1263,18 @@ public abstract class TaskAttemptImpl implements
|
|||
return slotMillisIncrement;
|
||||
}
|
||||
|
||||
private static JobCounterUpdateEvent createJobCounterUpdateEventTASucceeded(
|
||||
TaskAttemptImpl taskAttempt) {
|
||||
long slotMillis = computeSlotMillis(taskAttempt);
|
||||
TaskId taskId = taskAttempt.attemptId.getTaskId();
|
||||
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(taskId.getJobId());
|
||||
jce.addCounterUpdate(
|
||||
taskId.getTaskType() == TaskType.MAP ?
|
||||
JobCounter.SLOTS_MILLIS_MAPS : JobCounter.SLOTS_MILLIS_REDUCES,
|
||||
slotMillis);
|
||||
return jce;
|
||||
}
|
||||
|
||||
private static JobCounterUpdateEvent createJobCounterUpdateEventTAFailed(
|
||||
TaskAttemptImpl taskAttempt, boolean taskAlreadyCompleted) {
|
||||
TaskType taskType = taskAttempt.getID().getTaskId().getTaskType();
|
||||
|
@ -1210,6 +1344,26 @@ public abstract class TaskAttemptImpl implements
|
|||
return tauce;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void sendLaunchedEvents() {
|
||||
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptId.getTaskId()
|
||||
.getJobId());
|
||||
jce.addCounterUpdate(attemptId.getTaskId().getTaskType() == TaskType.MAP ?
|
||||
JobCounter.TOTAL_LAUNCHED_MAPS : JobCounter.TOTAL_LAUNCHED_REDUCES, 1);
|
||||
eventHandler.handle(jce);
|
||||
|
||||
LOG.info("TaskAttempt: [" + attemptId
|
||||
+ "] using containerId: [" + containerID + " on NM: ["
|
||||
+ containerMgrAddress + "]");
|
||||
TaskAttemptStartedEvent tase =
|
||||
new TaskAttemptStartedEvent(TypeConverter.fromYarn(attemptId),
|
||||
TypeConverter.fromYarn(attemptId.getTaskId().getTaskType()),
|
||||
launchTime, trackerName, httpPort, shufflePort, containerID,
|
||||
locality.toString(), avataar.toString());
|
||||
eventHandler.handle(
|
||||
new JobHistoryEvent(attemptId.getTaskId().getJobId(), tase));
|
||||
}
|
||||
|
||||
private WrappedProgressSplitsBlock getProgressSplitBlock() {
|
||||
readLock.lock();
|
||||
try {
|
||||
|
@ -1342,8 +1496,6 @@ public abstract class TaskAttemptImpl implements
|
|||
taskAttempt.containerNodeId.toString());
|
||||
taskAttempt.nodeHttpAddress = StringInterner.weakIntern(
|
||||
cEvent.getContainer().getNodeHttpAddress());
|
||||
taskAttempt.nodeRackName = RackResolver.resolve(
|
||||
taskAttempt.containerNodeId.getHost()).getNetworkLocation();
|
||||
taskAttempt.containerToken = cEvent.getContainer().getContainerToken();
|
||||
taskAttempt.assignedCapability = cEvent.getContainer().getResource();
|
||||
// this is a _real_ Task (classic Hadoop mapred flavor):
|
||||
|
@ -1354,32 +1506,18 @@ public abstract class TaskAttemptImpl implements
|
|||
taskAttempt.taskAttemptListener.registerPendingTask(
|
||||
taskAttempt.remoteTask, taskAttempt.jvmID);
|
||||
|
||||
taskAttempt.locality = Locality.OFF_SWITCH;
|
||||
if (taskAttempt.dataLocalHosts.size() > 0) {
|
||||
String cHost = taskAttempt.resolveHost(
|
||||
taskAttempt.containerNodeId.getHost());
|
||||
if (taskAttempt.dataLocalHosts.contains(cHost)) {
|
||||
taskAttempt.locality = Locality.NODE_LOCAL;
|
||||
}
|
||||
}
|
||||
if (taskAttempt.locality == Locality.OFF_SWITCH) {
|
||||
if (taskAttempt.dataLocalRacks.contains(taskAttempt.nodeRackName)) {
|
||||
taskAttempt.locality = Locality.RACK_LOCAL;
|
||||
}
|
||||
}
|
||||
taskAttempt.computeRackAndLocality();
|
||||
|
||||
//launch the container
|
||||
//create the container object to be launched for a given Task attempt
|
||||
ContainerLaunchContext launchContext = createContainerLaunchContext(
|
||||
cEvent.getApplicationACLs(), taskAttempt.containerID,
|
||||
taskAttempt.conf, taskAttempt.jobToken, taskAttempt.remoteTask,
|
||||
taskAttempt.oldJobId, taskAttempt.assignedCapability,
|
||||
taskAttempt.jvmID, taskAttempt.taskAttemptListener,
|
||||
taskAttempt.credentials);
|
||||
cEvent.getApplicationACLs(), taskAttempt.conf, taskAttempt.jobToken,
|
||||
taskAttempt.remoteTask, taskAttempt.oldJobId, taskAttempt.jvmID,
|
||||
taskAttempt.taskAttemptListener, taskAttempt.credentials);
|
||||
taskAttempt.eventHandler.handle(new ContainerRemoteLaunchEvent(
|
||||
taskAttempt.attemptId, taskAttempt.containerID,
|
||||
taskAttempt.containerMgrAddress, taskAttempt.containerToken,
|
||||
launchContext, taskAttempt.remoteTask));
|
||||
launchContext, taskAttempt.assignedCapability, taskAttempt.remoteTask));
|
||||
|
||||
// send event to speculator that our container needs are satisfied
|
||||
taskAttempt.eventHandler.handle
|
||||
|
@ -1471,27 +1609,7 @@ public abstract class TaskAttemptImpl implements
|
|||
// Costly?
|
||||
taskAttempt.trackerName = nodeHttpInetAddr.getHostName();
|
||||
taskAttempt.httpPort = nodeHttpInetAddr.getPort();
|
||||
JobCounterUpdateEvent jce =
|
||||
new JobCounterUpdateEvent(taskAttempt.attemptId.getTaskId()
|
||||
.getJobId());
|
||||
jce.addCounterUpdate(
|
||||
taskAttempt.attemptId.getTaskId().getTaskType() == TaskType.MAP ?
|
||||
JobCounter.TOTAL_LAUNCHED_MAPS: JobCounter.TOTAL_LAUNCHED_REDUCES
|
||||
, 1);
|
||||
taskAttempt.eventHandler.handle(jce);
|
||||
|
||||
LOG.info("TaskAttempt: [" + taskAttempt.attemptId
|
||||
+ "] using containerId: [" + taskAttempt.containerID + " on NM: ["
|
||||
+ taskAttempt.containerMgrAddress + "]");
|
||||
TaskAttemptStartedEvent tase =
|
||||
new TaskAttemptStartedEvent(TypeConverter.fromYarn(taskAttempt.attemptId),
|
||||
TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId().getTaskType()),
|
||||
taskAttempt.launchTime,
|
||||
nodeHttpInetAddr.getHostName(), nodeHttpInetAddr.getPort(),
|
||||
taskAttempt.shufflePort, taskAttempt.containerID,
|
||||
taskAttempt.locality.toString(), taskAttempt.avataar.toString());
|
||||
taskAttempt.eventHandler.handle
|
||||
(new JobHistoryEvent(taskAttempt.attemptId.getTaskId().getJobId(), tase));
|
||||
taskAttempt.sendLaunchedEvents();
|
||||
taskAttempt.eventHandler.handle
|
||||
(new SpeculatorEvent
|
||||
(taskAttempt.attemptId, true, taskAttempt.clock.getTime()));
|
||||
|
@ -1540,14 +1658,8 @@ public abstract class TaskAttemptImpl implements
|
|||
TaskAttemptEvent event) {
|
||||
//set the finish time
|
||||
taskAttempt.setFinishTime();
|
||||
long slotMillis = computeSlotMillis(taskAttempt);
|
||||
TaskId taskId = taskAttempt.attemptId.getTaskId();
|
||||
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(taskId.getJobId());
|
||||
jce.addCounterUpdate(
|
||||
taskId.getTaskType() == TaskType.MAP ?
|
||||
JobCounter.SLOTS_MILLIS_MAPS : JobCounter.SLOTS_MILLIS_REDUCES,
|
||||
slotMillis);
|
||||
taskAttempt.eventHandler.handle(jce);
|
||||
taskAttempt.eventHandler.handle(
|
||||
createJobCounterUpdateEventTASucceeded(taskAttempt));
|
||||
taskAttempt.logAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
|
||||
taskAttempt.eventHandler.handle(new TaskTAttemptEvent(
|
||||
taskAttempt.attemptId,
|
||||
|
@ -1585,6 +1697,18 @@ public abstract class TaskAttemptImpl implements
|
|||
}
|
||||
}
|
||||
|
||||
private static class RecoverTransition implements
|
||||
MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptStateInternal> {
|
||||
|
||||
@Override
|
||||
public TaskAttemptStateInternal transition(TaskAttemptImpl taskAttempt,
|
||||
TaskAttemptEvent event) {
|
||||
TaskAttemptRecoverEvent tare = (TaskAttemptRecoverEvent) event;
|
||||
return taskAttempt.recover(tare.getTaskAttemptInfo(),
|
||||
tare.getCommitter(), tare.getRecoverOutput());
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings({ "unchecked" })
|
||||
private void logAttemptFinishedEvent(TaskAttemptStateInternal state) {
|
||||
//Log finished events only if an attempt started.
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.mapreduce.v2.app.job.impl;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.EnumSet;
|
||||
|
@ -37,7 +38,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapreduce.Counters;
|
||||
import org.apache.hadoop.mapreduce.MRConfig;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
|
||||
|
@ -69,8 +70,10 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptCompletedEvent
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptRecoverEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskRecoverEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerFailedEvent;
|
||||
|
@ -152,6 +155,12 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
TaskEventType.T_SCHEDULE, new InitialScheduleTransition())
|
||||
.addTransition(TaskStateInternal.NEW, TaskStateInternal.KILLED,
|
||||
TaskEventType.T_KILL, new KillNewTransition())
|
||||
.addTransition(TaskStateInternal.NEW,
|
||||
EnumSet.of(TaskStateInternal.FAILED,
|
||||
TaskStateInternal.KILLED,
|
||||
TaskStateInternal.RUNNING,
|
||||
TaskStateInternal.SUCCEEDED),
|
||||
TaskEventType.T_RECOVER, new RecoverTransition())
|
||||
|
||||
// Transitions from SCHEDULED state
|
||||
//when the first attempt is launched, the task state is set to RUNNING
|
||||
|
@ -250,20 +259,16 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
|
||||
// By default, the next TaskAttempt number is zero. Changes during recovery
|
||||
protected int nextAttemptNumber = 0;
|
||||
private List<TaskAttemptInfo> taskAttemptsFromPreviousGeneration =
|
||||
new ArrayList<TaskAttemptInfo>();
|
||||
|
||||
private static final class RecoverdAttemptsComparator implements
|
||||
Comparator<TaskAttemptInfo> {
|
||||
// For sorting task attempts by completion time
|
||||
private static final Comparator<TaskAttemptInfo> TA_INFO_COMPARATOR =
|
||||
new Comparator<TaskAttemptInfo>() {
|
||||
@Override
|
||||
public int compare(TaskAttemptInfo attempt1, TaskAttemptInfo attempt2) {
|
||||
long diff = attempt1.getStartTime() - attempt2.getStartTime();
|
||||
public int compare(TaskAttemptInfo a, TaskAttemptInfo b) {
|
||||
long diff = a.getFinishTime() - b.getFinishTime();
|
||||
return diff == 0 ? 0 : (diff < 0 ? -1 : 1);
|
||||
}
|
||||
}
|
||||
|
||||
private static final RecoverdAttemptsComparator RECOVERED_ATTEMPTS_COMPARATOR =
|
||||
new RecoverdAttemptsComparator();
|
||||
};
|
||||
|
||||
@Override
|
||||
public TaskState getState() {
|
||||
|
@ -280,8 +285,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
TaskAttemptListener taskAttemptListener,
|
||||
Token<JobTokenIdentifier> jobToken,
|
||||
Credentials credentials, Clock clock,
|
||||
Map<TaskId, TaskInfo> completedTasksFromPreviousRun, int startCount,
|
||||
MRAppMetrics metrics, AppContext appContext) {
|
||||
int appAttemptId, MRAppMetrics metrics, AppContext appContext) {
|
||||
this.conf = conf;
|
||||
this.clock = clock;
|
||||
this.jobFile = remoteJobConfFile;
|
||||
|
@ -307,41 +311,15 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
this.encryptedShuffle = conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY,
|
||||
MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT);
|
||||
|
||||
// See if this is from a previous generation.
|
||||
if (completedTasksFromPreviousRun != null
|
||||
&& completedTasksFromPreviousRun.containsKey(taskId)) {
|
||||
// This task has TaskAttempts from previous generation. We have to replay
|
||||
// them.
|
||||
LOG.info("Task is from previous run " + taskId);
|
||||
TaskInfo taskInfo = completedTasksFromPreviousRun.get(taskId);
|
||||
Map<TaskAttemptID, TaskAttemptInfo> allAttempts =
|
||||
taskInfo.getAllTaskAttempts();
|
||||
taskAttemptsFromPreviousGeneration = new ArrayList<TaskAttemptInfo>();
|
||||
taskAttemptsFromPreviousGeneration.addAll(allAttempts.values());
|
||||
Collections.sort(taskAttemptsFromPreviousGeneration,
|
||||
RECOVERED_ATTEMPTS_COMPARATOR);
|
||||
}
|
||||
|
||||
if (taskAttemptsFromPreviousGeneration.isEmpty()) {
|
||||
// All the previous attempts are exhausted, now start with a new
|
||||
// generation.
|
||||
// This "this leak" is okay because the retained pointer is in an
|
||||
// instance variable.
|
||||
stateMachine = stateMachineFactory.make(this);
|
||||
|
||||
// All the new TaskAttemptIDs are generated based on MR
|
||||
// ApplicationAttemptID so that attempts from previous lives don't
|
||||
// over-step the current one. This assumes that a task won't have more
|
||||
// than 1000 attempts in its single generation, which is very reasonable.
|
||||
// Someone is nuts if he/she thinks he/she can live with 1000 TaskAttempts
|
||||
// and requires serious medical attention.
|
||||
nextAttemptNumber = (startCount - 1) * 1000;
|
||||
} else {
|
||||
// There are still some TaskAttempts from previous generation, use them
|
||||
nextAttemptNumber =
|
||||
taskAttemptsFromPreviousGeneration.remove(0).getAttemptId().getId();
|
||||
}
|
||||
|
||||
// This "this leak" is okay because the retained pointer is in an
|
||||
// instance variable.
|
||||
stateMachine = stateMachineFactory.make(this);
|
||||
nextAttemptNumber = (appAttemptId - 1) * 1000;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -600,14 +578,28 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
|
||||
// This is always called in the Write Lock
|
||||
private void addAndScheduleAttempt(Avataar avataar) {
|
||||
TaskAttempt attempt = createAttempt();
|
||||
((TaskAttemptImpl) attempt).setAvataar(avataar);
|
||||
TaskAttempt attempt = addAttempt(avataar);
|
||||
inProgressAttempts.add(attempt.getID());
|
||||
//schedule the nextAttemptNumber
|
||||
if (failedAttempts.size() > 0) {
|
||||
eventHandler.handle(new TaskAttemptEvent(attempt.getID(),
|
||||
TaskAttemptEventType.TA_RESCHEDULE));
|
||||
} else {
|
||||
eventHandler.handle(new TaskAttemptEvent(attempt.getID(),
|
||||
TaskAttemptEventType.TA_SCHEDULE));
|
||||
}
|
||||
}
|
||||
|
||||
private TaskAttemptImpl addAttempt(Avataar avataar) {
|
||||
TaskAttemptImpl attempt = createAttempt();
|
||||
attempt.setAvataar(avataar);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Created attempt " + attempt.getID());
|
||||
}
|
||||
switch (attempts.size()) {
|
||||
case 0:
|
||||
attempts = Collections.singletonMap(attempt.getID(), attempt);
|
||||
attempts = Collections.singletonMap(attempt.getID(),
|
||||
(TaskAttempt) attempt);
|
||||
break;
|
||||
|
||||
case 1:
|
||||
|
@ -623,24 +615,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
break;
|
||||
}
|
||||
|
||||
// Update nextATtemptNumber
|
||||
if (taskAttemptsFromPreviousGeneration.isEmpty()) {
|
||||
++nextAttemptNumber;
|
||||
} else {
|
||||
// There are still some TaskAttempts from previous generation, use them
|
||||
nextAttemptNumber =
|
||||
taskAttemptsFromPreviousGeneration.remove(0).getAttemptId().getId();
|
||||
}
|
||||
|
||||
inProgressAttempts.add(attempt.getID());
|
||||
//schedule the nextAttemptNumber
|
||||
if (failedAttempts.size() > 0) {
|
||||
eventHandler.handle(new TaskAttemptEvent(attempt.getID(),
|
||||
TaskAttemptEventType.TA_RESCHEDULE));
|
||||
} else {
|
||||
eventHandler.handle(new TaskAttemptEvent(attempt.getID(),
|
||||
TaskAttemptEventType.TA_SCHEDULE));
|
||||
}
|
||||
return attempt;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -705,6 +681,16 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
}
|
||||
}
|
||||
|
||||
private void sendTaskStartedEvent() {
|
||||
TaskStartedEvent tse = new TaskStartedEvent(
|
||||
TypeConverter.fromYarn(taskId), getLaunchTime(),
|
||||
TypeConverter.fromYarn(taskId.getTaskType()),
|
||||
getSplitsAsString());
|
||||
eventHandler
|
||||
.handle(new JobHistoryEvent(taskId.getJobId(), tse));
|
||||
historyTaskStartGenerated = true;
|
||||
}
|
||||
|
||||
private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task, TaskStateInternal taskState) {
|
||||
TaskFinishedEvent tfe =
|
||||
new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId),
|
||||
|
@ -740,6 +726,16 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
task.successfulAttempt = null;
|
||||
}
|
||||
|
||||
private void sendTaskSucceededEvents() {
|
||||
eventHandler.handle(new JobTaskEvent(taskId, TaskState.SUCCEEDED));
|
||||
LOG.info("Task succeeded with attempt " + successfulAttempt);
|
||||
if (historyTaskStartGenerated) {
|
||||
TaskFinishedEvent tfe = createTaskFinishedEvent(this,
|
||||
TaskStateInternal.SUCCEEDED);
|
||||
eventHandler.handle(new JobHistoryEvent(taskId.getJobId(), tfe));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return a String representation of the splits.
|
||||
*
|
||||
|
@ -751,6 +747,122 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
return "";
|
||||
}
|
||||
|
||||
/**
|
||||
* Recover a completed task from a previous application attempt
|
||||
* @param taskInfo recovered info about the task
|
||||
* @param recoverTaskOutput whether to recover task outputs
|
||||
* @return state of the task after recovery
|
||||
*/
|
||||
private TaskStateInternal recover(TaskInfo taskInfo,
|
||||
OutputCommitter committer, boolean recoverTaskOutput) {
|
||||
LOG.info("Recovering task " + taskId
|
||||
+ " from prior app attempt, status was " + taskInfo.getTaskStatus());
|
||||
|
||||
scheduledTime = taskInfo.getStartTime();
|
||||
sendTaskStartedEvent();
|
||||
Collection<TaskAttemptInfo> attemptInfos =
|
||||
taskInfo.getAllTaskAttempts().values();
|
||||
|
||||
if (attemptInfos.size() > 0) {
|
||||
metrics.launchedTask(this);
|
||||
}
|
||||
|
||||
// recover the attempts for this task in the order they finished
|
||||
// so task attempt completion events are ordered properly
|
||||
int savedNextAttemptNumber = nextAttemptNumber;
|
||||
ArrayList<TaskAttemptInfo> taInfos =
|
||||
new ArrayList<TaskAttemptInfo>(taskInfo.getAllTaskAttempts().values());
|
||||
Collections.sort(taInfos, TA_INFO_COMPARATOR);
|
||||
for (TaskAttemptInfo taInfo : taInfos) {
|
||||
nextAttemptNumber = taInfo.getAttemptId().getId();
|
||||
TaskAttemptImpl attempt = addAttempt(Avataar.VIRGIN);
|
||||
// handle the recovery inline so attempts complete before task does
|
||||
attempt.handle(new TaskAttemptRecoverEvent(attempt.getID(), taInfo,
|
||||
committer, recoverTaskOutput));
|
||||
finishedAttempts.add(attempt.getID());
|
||||
TaskAttemptCompletionEventStatus taces = null;
|
||||
TaskAttemptState attemptState = attempt.getState();
|
||||
switch (attemptState) {
|
||||
case FAILED:
|
||||
taces = TaskAttemptCompletionEventStatus.FAILED;
|
||||
break;
|
||||
case KILLED:
|
||||
taces = TaskAttemptCompletionEventStatus.KILLED;
|
||||
break;
|
||||
case SUCCEEDED:
|
||||
taces = TaskAttemptCompletionEventStatus.SUCCEEDED;
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException(
|
||||
"Unexpected attempt state during recovery: " + attemptState);
|
||||
}
|
||||
if (attemptState == TaskAttemptState.FAILED) {
|
||||
failedAttempts.add(attempt.getID());
|
||||
if (failedAttempts.size() >= maxAttempts) {
|
||||
taces = TaskAttemptCompletionEventStatus.TIPFAILED;
|
||||
}
|
||||
}
|
||||
|
||||
// don't clobber the successful attempt completion event
|
||||
// TODO: this shouldn't be necessary after MAPREDUCE-4330
|
||||
if (successfulAttempt == null) {
|
||||
handleTaskAttemptCompletion(attempt.getID(), taces);
|
||||
if (attemptState == TaskAttemptState.SUCCEEDED) {
|
||||
successfulAttempt = attempt.getID();
|
||||
}
|
||||
}
|
||||
}
|
||||
nextAttemptNumber = savedNextAttemptNumber;
|
||||
|
||||
TaskStateInternal taskState = TaskStateInternal.valueOf(
|
||||
taskInfo.getTaskStatus());
|
||||
switch (taskState) {
|
||||
case SUCCEEDED:
|
||||
if (successfulAttempt != null) {
|
||||
sendTaskSucceededEvents();
|
||||
} else {
|
||||
LOG.info("Missing successful attempt for task " + taskId
|
||||
+ ", recovering as RUNNING");
|
||||
// there must have been a fetch failure and the retry wasn't complete
|
||||
taskState = TaskStateInternal.RUNNING;
|
||||
metrics.runningTask(this);
|
||||
addAndScheduleAttempt(Avataar.VIRGIN);
|
||||
}
|
||||
break;
|
||||
case FAILED:
|
||||
case KILLED:
|
||||
{
|
||||
if (taskState == TaskStateInternal.KILLED && attemptInfos.size() == 0) {
|
||||
metrics.endWaitingTask(this);
|
||||
}
|
||||
TaskFailedEvent tfe = new TaskFailedEvent(taskInfo.getTaskId(),
|
||||
taskInfo.getFinishTime(), taskInfo.getTaskType(),
|
||||
taskInfo.getError(), taskInfo.getTaskStatus(),
|
||||
taskInfo.getFailedDueToAttemptId(), taskInfo.getCounters());
|
||||
eventHandler.handle(new JobHistoryEvent(taskId.getJobId(), tfe));
|
||||
eventHandler.handle(
|
||||
new JobTaskEvent(taskId, getExternalState(taskState)));
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw new java.lang.AssertionError("Unexpected recovered task state: "
|
||||
+ taskState);
|
||||
}
|
||||
|
||||
return taskState;
|
||||
}
|
||||
|
||||
private static class RecoverTransition
|
||||
implements MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
|
||||
|
||||
@Override
|
||||
public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
|
||||
TaskRecoverEvent tre = (TaskRecoverEvent) event;
|
||||
return task.recover(tre.getTaskInfo(), tre.getOutputCommitter(),
|
||||
tre.getRecoverTaskOutput());
|
||||
}
|
||||
}
|
||||
|
||||
private static class InitialScheduleTransition
|
||||
implements SingleArcTransition<TaskImpl, TaskEvent> {
|
||||
|
||||
|
@ -758,13 +870,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
public void transition(TaskImpl task, TaskEvent event) {
|
||||
task.addAndScheduleAttempt(Avataar.VIRGIN);
|
||||
task.scheduledTime = task.clock.getTime();
|
||||
TaskStartedEvent tse = new TaskStartedEvent(
|
||||
TypeConverter.fromYarn(task.taskId), task.getLaunchTime(),
|
||||
TypeConverter.fromYarn(task.taskId.getTaskType()),
|
||||
task.getSplitsAsString());
|
||||
task.eventHandler
|
||||
.handle(new JobHistoryEvent(task.taskId.getJobId(), tse));
|
||||
task.historyTaskStartGenerated = true;
|
||||
task.sendTaskStartedEvent();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -818,16 +924,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
task.finishedAttempts.add(taskAttemptId);
|
||||
task.inProgressAttempts.remove(taskAttemptId);
|
||||
task.successfulAttempt = taskAttemptId;
|
||||
task.eventHandler.handle(new JobTaskEvent(
|
||||
task.taskId, TaskState.SUCCEEDED));
|
||||
LOG.info("Task succeeded with attempt " + task.successfulAttempt);
|
||||
// issue kill to all other attempts
|
||||
if (task.historyTaskStartGenerated) {
|
||||
TaskFinishedEvent tfe = createTaskFinishedEvent(task,
|
||||
TaskStateInternal.SUCCEEDED);
|
||||
task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
|
||||
tfe));
|
||||
}
|
||||
task.sendTaskSucceededEvents();
|
||||
for (TaskAttempt attempt : task.attempts.values()) {
|
||||
if (attempt.getID() != task.successfulAttempt &&
|
||||
// This is okay because it can only talk us out of sending a
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.api.records.ContainerToken;
|
|||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.service.AbstractService;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.ProtoUtils;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
|
@ -150,10 +151,14 @@ public class ContainerLauncherImpl extends AbstractService implements
|
|||
ContainerLaunchContext containerLaunchContext =
|
||||
event.getContainer();
|
||||
|
||||
org.apache.hadoop.yarn.api.records.Container container =
|
||||
BuilderUtils.newContainer(containerID, null, null,
|
||||
event.getResource(), null, containerToken);
|
||||
// Now launch the actual container
|
||||
StartContainerRequest startRequest = Records
|
||||
.newRecord(StartContainerRequest.class);
|
||||
startRequest.setContainerLaunchContext(containerLaunchContext);
|
||||
startRequest.setContainer(container);
|
||||
StartContainerResponse response = proxy.startContainer(startRequest);
|
||||
|
||||
ByteBuffer portInfo = response
|
||||
|
|
|
@ -23,26 +23,34 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
|||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerToken;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
|
||||
public class ContainerRemoteLaunchEvent extends ContainerLauncherEvent {
|
||||
|
||||
private final ContainerLaunchContext container;
|
||||
private final Task task;
|
||||
private final Resource resource;
|
||||
|
||||
public ContainerRemoteLaunchEvent(TaskAttemptId taskAttemptID,
|
||||
ContainerId containerID, String containerMgrAddress,
|
||||
ContainerToken containerToken,
|
||||
ContainerLaunchContext containerLaunchContext, Task remoteTask) {
|
||||
ContainerLaunchContext containerLaunchContext, Resource resource,
|
||||
Task remoteTask) {
|
||||
super(taskAttemptID, containerID, containerMgrAddress, containerToken,
|
||||
ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH);
|
||||
this.container = containerLaunchContext;
|
||||
this.task = remoteTask;
|
||||
this.resource = resource;
|
||||
}
|
||||
|
||||
public ContainerLaunchContext getContainer() {
|
||||
return this.container;
|
||||
}
|
||||
|
||||
public Resource getResource() {
|
||||
return this.resource;
|
||||
}
|
||||
|
||||
public Task getRemoteTask() {
|
||||
return this.task;
|
||||
}
|
||||
|
|
|
@ -123,7 +123,7 @@ public class LocalContainerAllocator extends RMCommunicator
|
|||
// This can happen if the RM has been restarted. If it is in that state,
|
||||
// this application must clean itself up.
|
||||
eventHandler.handle(new JobEvent(this.getJob().getID(),
|
||||
JobEventType.INTERNAL_ERROR));
|
||||
JobEventType.JOB_AM_REBOOT));
|
||||
throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
|
||||
this.getContext().getApplicationID());
|
||||
}
|
||||
|
|
|
@ -1,39 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app.recover;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
|
||||
public interface Recovery {
|
||||
|
||||
Dispatcher getDispatcher();
|
||||
|
||||
Clock getClock();
|
||||
|
||||
Map<TaskId, TaskInfo> getCompletedTasks();
|
||||
|
||||
List<AMInfo> getAMInfos();
|
||||
}
|
|
@ -1,480 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.mapreduce.v2.app.recover;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.mapred.JobConf;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
||||
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;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.Phase;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
||||
import org.apache.hadoop.mapreduce.v2.app.ControlledClock;
|
||||
import org.apache.hadoop.mapreduce.v2.app.commit.CommitterTaskAbortEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
|
||||
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.event.Event;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.service.CompositeService;
|
||||
import org.apache.hadoop.yarn.service.Service;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
|
||||
/*
|
||||
* Recovers the completed tasks from the previous life of Application Master.
|
||||
* The completed tasks are deciphered from the history file of the previous life.
|
||||
* Recovery service intercepts and replay the events for completed tasks.
|
||||
* While recovery is in progress, the scheduling of new tasks are delayed by
|
||||
* buffering the task schedule events.
|
||||
* The recovery service controls the clock while recovery is in progress.
|
||||
*/
|
||||
|
||||
//TODO:
|
||||
//task cleanup for all non completed tasks
|
||||
public class RecoveryService extends CompositeService implements Recovery {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(RecoveryService.class);
|
||||
|
||||
private final ApplicationAttemptId applicationAttemptId;
|
||||
private final OutputCommitter committer;
|
||||
private final boolean newApiCommitter;
|
||||
private final Dispatcher dispatcher;
|
||||
private final ControlledClock clock;
|
||||
|
||||
private JobInfo jobInfo = null;
|
||||
private final Map<TaskId, TaskInfo> completedTasks =
|
||||
new HashMap<TaskId, TaskInfo>();
|
||||
|
||||
private final List<TaskEvent> pendingTaskScheduleEvents =
|
||||
new ArrayList<TaskEvent>();
|
||||
|
||||
private volatile boolean recoveryMode = false;
|
||||
|
||||
public RecoveryService(ApplicationAttemptId applicationAttemptId,
|
||||
Clock clock, OutputCommitter committer, boolean newApiCommitter) {
|
||||
super("RecoveringDispatcher");
|
||||
this.applicationAttemptId = applicationAttemptId;
|
||||
this.committer = committer;
|
||||
this.newApiCommitter = newApiCommitter;
|
||||
this.dispatcher = createRecoveryDispatcher();
|
||||
this.clock = new ControlledClock(clock);
|
||||
addService((Service) dispatcher);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(Configuration conf) {
|
||||
super.init(conf);
|
||||
// parse the history file
|
||||
try {
|
||||
parse();
|
||||
} catch (Exception e) {
|
||||
LOG.warn(e);
|
||||
LOG.warn("Could not parse the old history file. Aborting recovery. "
|
||||
+ "Starting afresh.", e);
|
||||
}
|
||||
if (completedTasks.size() > 0) {
|
||||
recoveryMode = true;
|
||||
LOG.info("SETTING THE RECOVERY MODE TO TRUE. NO OF COMPLETED TASKS "
|
||||
+ "TO RECOVER " + completedTasks.size());
|
||||
LOG.info("Job launch time " + jobInfo.getLaunchTime());
|
||||
clock.setTime(jobInfo.getLaunchTime());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Dispatcher getDispatcher() {
|
||||
return dispatcher;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Clock getClock() {
|
||||
return clock;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<TaskId, TaskInfo> getCompletedTasks() {
|
||||
return completedTasks;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AMInfo> getAMInfos() {
|
||||
if (jobInfo == null || jobInfo.getAMInfos() == null) {
|
||||
return new LinkedList<AMInfo>();
|
||||
}
|
||||
List<AMInfo> amInfos = new LinkedList<AMInfo>();
|
||||
for (org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo jhAmInfo : jobInfo
|
||||
.getAMInfos()) {
|
||||
AMInfo amInfo =
|
||||
MRBuilderUtils.newAMInfo(jhAmInfo.getAppAttemptId(),
|
||||
jhAmInfo.getStartTime(), jhAmInfo.getContainerId(),
|
||||
jhAmInfo.getNodeManagerHost(), jhAmInfo.getNodeManagerPort(),
|
||||
jhAmInfo.getNodeManagerHttpPort());
|
||||
|
||||
amInfos.add(amInfo);
|
||||
}
|
||||
return amInfos;
|
||||
}
|
||||
|
||||
private void parse() throws IOException {
|
||||
FSDataInputStream in =
|
||||
getPreviousJobHistoryFileStream(getConfig(), applicationAttemptId);
|
||||
JobHistoryParser parser = new JobHistoryParser(in);
|
||||
jobInfo = parser.parse();
|
||||
Exception parseException = parser.getParseException();
|
||||
if (parseException != null) {
|
||||
LOG.info("Got an error parsing job-history file" +
|
||||
", ignoring incomplete events.", parseException);
|
||||
}
|
||||
Map<org.apache.hadoop.mapreduce.TaskID, TaskInfo> taskInfos = jobInfo
|
||||
.getAllTasks();
|
||||
for (TaskInfo taskInfo : taskInfos.values()) {
|
||||
if (TaskState.SUCCEEDED.toString().equals(taskInfo.getTaskStatus())) {
|
||||
Iterator<Entry<TaskAttemptID, TaskAttemptInfo>> taskAttemptIterator =
|
||||
taskInfo.getAllTaskAttempts().entrySet().iterator();
|
||||
while (taskAttemptIterator.hasNext()) {
|
||||
Map.Entry<TaskAttemptID, TaskAttemptInfo> currentEntry = taskAttemptIterator.next();
|
||||
if (!jobInfo.getAllCompletedTaskAttempts().containsKey(currentEntry.getKey())) {
|
||||
taskAttemptIterator.remove();
|
||||
}
|
||||
}
|
||||
completedTasks
|
||||
.put(TypeConverter.toYarn(taskInfo.getTaskId()), taskInfo);
|
||||
LOG.info("Read from history task "
|
||||
+ TypeConverter.toYarn(taskInfo.getTaskId()));
|
||||
}
|
||||
}
|
||||
LOG.info("Read completed tasks from history "
|
||||
+ completedTasks.size());
|
||||
}
|
||||
|
||||
public static FSDataInputStream getPreviousJobHistoryFileStream(
|
||||
Configuration conf, ApplicationAttemptId applicationAttemptId)
|
||||
throws IOException {
|
||||
FSDataInputStream in = null;
|
||||
Path historyFile = null;
|
||||
String jobId =
|
||||
TypeConverter.fromYarn(applicationAttemptId.getApplicationId())
|
||||
.toString();
|
||||
String jobhistoryDir =
|
||||
JobHistoryUtils.getConfiguredHistoryStagingDirPrefix(conf, jobId);
|
||||
Path histDirPath =
|
||||
FileContext.getFileContext(conf).makeQualified(new Path(jobhistoryDir));
|
||||
LOG.info("Trying file " + histDirPath.toString());
|
||||
FileContext fc = FileContext.getFileContext(histDirPath.toUri(), conf);
|
||||
// read the previous history file
|
||||
historyFile =
|
||||
fc.makeQualified(JobHistoryUtils.getStagingJobHistoryFile(histDirPath,
|
||||
jobId, (applicationAttemptId.getAttemptId() - 1)));
|
||||
LOG.info("History file is at " + historyFile);
|
||||
in = fc.open(historyFile);
|
||||
return in;
|
||||
}
|
||||
|
||||
protected Dispatcher createRecoveryDispatcher() {
|
||||
return new RecoveryDispatcher();
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
class RecoveryDispatcher extends AsyncDispatcher {
|
||||
private final EventHandler actualHandler;
|
||||
private final EventHandler handler;
|
||||
|
||||
RecoveryDispatcher() {
|
||||
super();
|
||||
actualHandler = super.getEventHandler();
|
||||
handler = new InterceptingEventHandler(actualHandler);
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public void dispatch(Event event) {
|
||||
if (recoveryMode) {
|
||||
if (event.getType() == TaskAttemptEventType.TA_CONTAINER_LAUNCHED) {
|
||||
TaskAttemptInfo attInfo = getTaskAttemptInfo(((TaskAttemptEvent) event)
|
||||
.getTaskAttemptID());
|
||||
LOG.info("Recovered Attempt start time " + attInfo.getStartTime());
|
||||
clock.setTime(attInfo.getStartTime());
|
||||
|
||||
} else if (event.getType() == TaskAttemptEventType.TA_DONE
|
||||
|| event.getType() == TaskAttemptEventType.TA_FAILMSG
|
||||
|| event.getType() == TaskAttemptEventType.TA_KILL) {
|
||||
TaskAttemptInfo attInfo = getTaskAttemptInfo(((TaskAttemptEvent) event)
|
||||
.getTaskAttemptID());
|
||||
LOG.info("Recovered Attempt finish time " + attInfo.getFinishTime());
|
||||
clock.setTime(attInfo.getFinishTime());
|
||||
}
|
||||
|
||||
else if (event.getType() == TaskEventType.T_ATTEMPT_FAILED
|
||||
|| event.getType() == TaskEventType.T_ATTEMPT_KILLED
|
||||
|| event.getType() == TaskEventType.T_ATTEMPT_SUCCEEDED) {
|
||||
TaskTAttemptEvent tEvent = (TaskTAttemptEvent) event;
|
||||
LOG.info("Recovered Task attempt " + tEvent.getTaskAttemptID());
|
||||
TaskInfo taskInfo = completedTasks.get(tEvent.getTaskAttemptID()
|
||||
.getTaskId());
|
||||
taskInfo.getAllTaskAttempts().remove(
|
||||
TypeConverter.fromYarn(tEvent.getTaskAttemptID()));
|
||||
// remove the task info from completed tasks if all attempts are
|
||||
// recovered
|
||||
if (taskInfo.getAllTaskAttempts().size() == 0) {
|
||||
completedTasks.remove(tEvent.getTaskAttemptID().getTaskId());
|
||||
// checkForRecoveryComplete
|
||||
LOG.info("CompletedTasks() " + completedTasks.size());
|
||||
if (completedTasks.size() == 0) {
|
||||
recoveryMode = false;
|
||||
clock.reset();
|
||||
LOG.info("Setting the recovery mode to false. " +
|
||||
"Recovery is complete!");
|
||||
|
||||
// send all pending tasks schedule events
|
||||
for (TaskEvent tEv : pendingTaskScheduleEvents) {
|
||||
actualHandler.handle(tEv);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
realDispatch(event);
|
||||
}
|
||||
|
||||
public void realDispatch(Event event) {
|
||||
super.dispatch(event);
|
||||
}
|
||||
|
||||
@Override
|
||||
public EventHandler getEventHandler() {
|
||||
return handler;
|
||||
}
|
||||
}
|
||||
|
||||
private TaskAttemptInfo getTaskAttemptInfo(TaskAttemptId id) {
|
||||
TaskInfo taskInfo = completedTasks.get(id.getTaskId());
|
||||
return taskInfo.getAllTaskAttempts().get(TypeConverter.fromYarn(id));
|
||||
}
|
||||
|
||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||
private class InterceptingEventHandler implements EventHandler {
|
||||
EventHandler actualHandler;
|
||||
|
||||
InterceptingEventHandler(EventHandler actualHandler) {
|
||||
this.actualHandler = actualHandler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle(Event event) {
|
||||
if (!recoveryMode) {
|
||||
// delegate to the dispatcher one
|
||||
actualHandler.handle(event);
|
||||
return;
|
||||
}
|
||||
|
||||
else if (event.getType() == TaskEventType.T_SCHEDULE) {
|
||||
TaskEvent taskEvent = (TaskEvent) event;
|
||||
// delay the scheduling of new tasks till previous ones are recovered
|
||||
if (completedTasks.get(taskEvent.getTaskID()) == null) {
|
||||
LOG.debug("Adding to pending task events "
|
||||
+ taskEvent.getTaskID());
|
||||
pendingTaskScheduleEvents.add(taskEvent);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
else if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
|
||||
TaskAttemptId aId = ((ContainerAllocatorEvent) event).getAttemptID();
|
||||
TaskAttemptInfo attInfo = getTaskAttemptInfo(aId);
|
||||
LOG.debug("CONTAINER_REQ " + aId);
|
||||
sendAssignedEvent(aId, attInfo);
|
||||
return;
|
||||
}
|
||||
|
||||
else if (event.getType() == CommitterEventType.TASK_ABORT) {
|
||||
TaskAttemptId aId = ((CommitterTaskAbortEvent) event).getAttemptID();
|
||||
LOG.debug("TASK_CLEAN");
|
||||
actualHandler.handle(new TaskAttemptEvent(aId,
|
||||
TaskAttemptEventType.TA_CLEANUP_DONE));
|
||||
return;
|
||||
}
|
||||
|
||||
else if (event.getType() == ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH) {
|
||||
TaskAttemptId aId = ((ContainerRemoteLaunchEvent) event)
|
||||
.getTaskAttemptID();
|
||||
TaskAttemptInfo attInfo = getTaskAttemptInfo(aId);
|
||||
actualHandler.handle(new TaskAttemptContainerLaunchedEvent(aId,
|
||||
attInfo.getShufflePort()));
|
||||
// send the status update event
|
||||
sendStatusUpdateEvent(aId, attInfo);
|
||||
|
||||
TaskAttemptState state = TaskAttemptState.valueOf(attInfo.getTaskStatus());
|
||||
switch (state) {
|
||||
case SUCCEEDED:
|
||||
//recover the task output
|
||||
|
||||
// check the committer type and construct corresponding context
|
||||
TaskAttemptContext taskContext = null;
|
||||
if(newApiCommitter) {
|
||||
taskContext = new TaskAttemptContextImpl(getConfig(),
|
||||
attInfo.getAttemptId());
|
||||
} else {
|
||||
taskContext = new org.apache.hadoop.mapred.TaskAttemptContextImpl(new JobConf(getConfig()),
|
||||
TypeConverter.fromYarn(aId));
|
||||
}
|
||||
|
||||
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));
|
||||
}
|
||||
|
||||
// send the done event
|
||||
LOG.info("Sending done event to recovered attempt " + aId);
|
||||
actualHandler.handle(new TaskAttemptEvent(aId,
|
||||
TaskAttemptEventType.TA_DONE));
|
||||
break;
|
||||
case KILLED:
|
||||
LOG.info("Sending kill event to recovered attempt " + aId);
|
||||
actualHandler.handle(new TaskAttemptEvent(aId,
|
||||
TaskAttemptEventType.TA_KILL));
|
||||
break;
|
||||
default:
|
||||
LOG.info("Sending fail event to recovered attempt " + aId);
|
||||
actualHandler.handle(new TaskAttemptEvent(aId,
|
||||
TaskAttemptEventType.TA_FAILMSG));
|
||||
break;
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
else if (event.getType() ==
|
||||
ContainerLauncher.EventType.CONTAINER_REMOTE_CLEANUP) {
|
||||
TaskAttemptId aId = ((ContainerLauncherEvent) event)
|
||||
.getTaskAttemptID();
|
||||
actualHandler.handle(
|
||||
new TaskAttemptEvent(aId,
|
||||
TaskAttemptEventType.TA_CONTAINER_CLEANED));
|
||||
return;
|
||||
}
|
||||
|
||||
// delegate to the actual handler
|
||||
actualHandler.handle(event);
|
||||
}
|
||||
|
||||
private void sendStatusUpdateEvent(TaskAttemptId yarnAttemptID,
|
||||
TaskAttemptInfo attemptInfo) {
|
||||
LOG.info("Sending status update event to " + yarnAttemptID);
|
||||
TaskAttemptStatus taskAttemptStatus = new TaskAttemptStatus();
|
||||
taskAttemptStatus.id = yarnAttemptID;
|
||||
taskAttemptStatus.progress = 1.0f;
|
||||
taskAttemptStatus.stateString = attemptInfo.getTaskStatus();
|
||||
// taskAttemptStatus.outputSize = attemptInfo.getOutputSize();
|
||||
taskAttemptStatus.phase = Phase.CLEANUP;
|
||||
org.apache.hadoop.mapreduce.Counters cntrs = attemptInfo.getCounters();
|
||||
if (cntrs == null) {
|
||||
taskAttemptStatus.counters = null;
|
||||
} else {
|
||||
taskAttemptStatus.counters = cntrs;
|
||||
}
|
||||
actualHandler.handle(new TaskAttemptStatusUpdateEvent(
|
||||
taskAttemptStatus.id, taskAttemptStatus));
|
||||
}
|
||||
|
||||
private void sendAssignedEvent(TaskAttemptId yarnAttemptID,
|
||||
TaskAttemptInfo attemptInfo) {
|
||||
LOG.info("Sending assigned event to " + yarnAttemptID);
|
||||
ContainerId cId = attemptInfo.getContainerId();
|
||||
|
||||
NodeId nodeId =
|
||||
ConverterUtils.toNodeId(attemptInfo.getHostname() + ":"
|
||||
+ attemptInfo.getPort());
|
||||
// Resource/Priority/ApplicationACLs are only needed while launching the
|
||||
// container on an NM, these are already completed tasks, so setting them
|
||||
// to null
|
||||
Container container = BuilderUtils.newContainer(cId, nodeId,
|
||||
attemptInfo.getTrackerName() + ":" + attemptInfo.getHttpPort(),
|
||||
null, null, null);
|
||||
actualHandler.handle(new TaskAttemptContainerAssignedEvent(yarnAttemptID,
|
||||
container, null));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -574,7 +574,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
// This can happen if the RM has been restarted. If it is in that state,
|
||||
// this application must clean itself up.
|
||||
eventHandler.handle(new JobEvent(this.getJob().getID(),
|
||||
JobEventType.INTERNAL_ERROR));
|
||||
JobEventType.JOB_AM_REBOOT));
|
||||
throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
|
||||
this.getContext().getApplicationID());
|
||||
}
|
||||
|
|
|
@ -33,6 +33,9 @@ import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
|||
import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
|
||||
import org.apache.hadoop.yarn.webapp.SubView;
|
||||
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
|
||||
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
|
||||
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
|
||||
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
|
||||
|
@ -53,7 +56,7 @@ public class TaskPage extends AppView {
|
|||
h2($(TITLE));
|
||||
return;
|
||||
}
|
||||
html.
|
||||
TBODY<TABLE<Hamlet>> tbody = html.
|
||||
table("#attempts").
|
||||
thead().
|
||||
tr().
|
||||
|
@ -65,7 +68,8 @@ public class TaskPage extends AppView {
|
|||
th(".tsh", "Started").
|
||||
th(".tsh", "Finished").
|
||||
th(".tsh", "Elapsed").
|
||||
th(".note", "Note")._()._();
|
||||
th(".note", "Note")._()._().
|
||||
tbody();
|
||||
// Write all the data into a JavaScript array of arrays for JQuery
|
||||
// DataTables to display
|
||||
StringBuilder attemptsTableData = new StringBuilder("[\n");
|
||||
|
@ -105,6 +109,9 @@ public class TaskPage extends AppView {
|
|||
attemptsTableData.append("]");
|
||||
html.script().$type("text/javascript").
|
||||
_("var attemptsTableData=" + attemptsTableData)._();
|
||||
|
||||
tbody._()._();
|
||||
|
||||
}
|
||||
|
||||
protected boolean isValidRequest() {
|
||||
|
|
|
@ -414,7 +414,8 @@ public class MRApp extends MRAppMaster {
|
|||
Job newJob = new TestJob(getJobId(), getAttemptID(), conf,
|
||||
getDispatcher().getEventHandler(),
|
||||
getTaskAttemptListener(), getContext().getClock(),
|
||||
isNewApiCommitter(), currentUser.getUserName(), getContext(),
|
||||
getCommitter(), isNewApiCommitter(),
|
||||
currentUser.getUserName(), getContext(),
|
||||
forcedState, diagnostic);
|
||||
((AppContext) getContext()).getAllJobs().put(newJob.getID(), newJob);
|
||||
|
||||
|
@ -648,12 +649,13 @@ public class MRApp extends MRAppMaster {
|
|||
public TestJob(JobId jobId, ApplicationAttemptId applicationAttemptId,
|
||||
Configuration conf, EventHandler eventHandler,
|
||||
TaskAttemptListener taskAttemptListener, Clock clock,
|
||||
boolean newApiCommitter, String user, AppContext appContext,
|
||||
OutputCommitter committer, boolean newApiCommitter,
|
||||
String user, AppContext appContext,
|
||||
JobStateInternal forcedState, String diagnostic) {
|
||||
super(jobId, getApplicationAttemptId(applicationId, getStartCount()),
|
||||
conf, eventHandler, taskAttemptListener,
|
||||
new JobTokenSecretManager(), new Credentials(), clock,
|
||||
getCompletedTaskFromPreviousRun(), metrics,
|
||||
getCompletedTaskFromPreviousRun(), metrics, committer,
|
||||
newApiCommitter, user, System.currentTimeMillis(), getAllAMInfos(),
|
||||
appContext, forcedState, diagnostic);
|
||||
|
||||
|
|
|
@ -18,10 +18,21 @@
|
|||
|
||||
package org.apache.hadoop.mapreduce.v2.app;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.atLeast;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
|
@ -31,36 +42,66 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapreduce.Counters;
|
||||
import org.apache.hadoop.mapreduce.JobCounter;
|
||||
import org.apache.hadoop.mapreduce.JobID;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
||||
import org.apache.hadoop.mapreduce.TaskID;
|
||||
import org.apache.hadoop.mapreduce.TaskType;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.Event;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.EventType;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
|
||||
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
|
||||
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
|
||||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
|
||||
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.TaskAttemptState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskRecoverEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.impl.MapTaskImpl;
|
||||
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
|
||||
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.hadoop.yarn.Clock;
|
||||
import org.apache.hadoop.yarn.ClusterInfo;
|
||||
import org.apache.hadoop.yarn.SystemClock;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.junit.Test;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public class TestRecovery {
|
||||
|
@ -75,6 +116,7 @@ public class TestRecovery {
|
|||
private Text val1 = new Text("val1");
|
||||
private Text val2 = new Text("val2");
|
||||
|
||||
|
||||
/**
|
||||
* AM with 2 maps and 1 reduce. For 1st map, one attempt fails, one attempt
|
||||
* completely disappears because of failed launch, one attempt gets killed and
|
||||
|
@ -1011,6 +1053,423 @@ public class TestRecovery {
|
|||
app.verifyCompleted();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecoverySuccessAttempt() {
|
||||
LOG.info("--- START: testRecoverySuccessAttempt ---");
|
||||
|
||||
long clusterTimestamp = System.currentTimeMillis();
|
||||
EventHandler mockEventHandler = mock(EventHandler.class);
|
||||
MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
||||
mockEventHandler);
|
||||
|
||||
TaskId taskId = recoverMapTask.getID();
|
||||
JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
||||
TaskID taskID = new TaskID(jobID,
|
||||
org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
||||
|
||||
//Mock up the TaskAttempts
|
||||
Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
||||
new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
||||
|
||||
TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
||||
TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
||||
TaskAttemptState.SUCCEEDED);
|
||||
mockTaskAttempts.put(taId1, mockTAinfo1);
|
||||
|
||||
TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
||||
TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
||||
TaskAttemptState.FAILED);
|
||||
mockTaskAttempts.put(taId2, mockTAinfo2);
|
||||
|
||||
OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
||||
TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
||||
when(mockTaskInfo.getTaskStatus()).thenReturn("SUCCEEDED");
|
||||
when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
||||
when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
||||
|
||||
recoverMapTask.handle(
|
||||
new TaskRecoverEvent(taskId, mockTaskInfo,mockCommitter, true));
|
||||
|
||||
ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
||||
verify(mockEventHandler,atLeast(1)).handle(
|
||||
(org.apache.hadoop.yarn.event.Event) arg.capture());
|
||||
|
||||
Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
||||
new HashMap<TaskAttemptID, TaskAttemptState>();
|
||||
finalAttemptStates.put(taId1, TaskAttemptState.SUCCEEDED);
|
||||
finalAttemptStates.put(taId2, TaskAttemptState.FAILED);
|
||||
|
||||
List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
||||
jobHistoryEvents.add(EventType.TASK_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_FINISHED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
||||
jobHistoryEvents.add(EventType.TASK_FINISHED);
|
||||
recoveryChecker(recoverMapTask, TaskState.SUCCEEDED, finalAttemptStates,
|
||||
arg, jobHistoryEvents, 2L, 1L);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecoveryAllFailAttempts() {
|
||||
LOG.info("--- START: testRecoveryAllFailAttempts ---");
|
||||
|
||||
long clusterTimestamp = System.currentTimeMillis();
|
||||
EventHandler mockEventHandler = mock(EventHandler.class);
|
||||
MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
||||
mockEventHandler);
|
||||
|
||||
TaskId taskId = recoverMapTask.getID();
|
||||
JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
||||
TaskID taskID = new TaskID(jobID,
|
||||
org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
||||
|
||||
//Mock up the TaskAttempts
|
||||
Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
||||
new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
||||
|
||||
TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
||||
TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
||||
TaskAttemptState.FAILED);
|
||||
mockTaskAttempts.put(taId1, mockTAinfo1);
|
||||
|
||||
TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
||||
TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
||||
TaskAttemptState.FAILED);
|
||||
mockTaskAttempts.put(taId2, mockTAinfo2);
|
||||
|
||||
OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
||||
|
||||
TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
||||
when(mockTaskInfo.getTaskStatus()).thenReturn("FAILED");
|
||||
when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
||||
when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
||||
|
||||
recoverMapTask.handle(
|
||||
new TaskRecoverEvent(taskId, mockTaskInfo, mockCommitter, true));
|
||||
|
||||
ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
||||
verify(mockEventHandler,atLeast(1)).handle(
|
||||
(org.apache.hadoop.yarn.event.Event) arg.capture());
|
||||
|
||||
Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
||||
new HashMap<TaskAttemptID, TaskAttemptState>();
|
||||
finalAttemptStates.put(taId1, TaskAttemptState.FAILED);
|
||||
finalAttemptStates.put(taId2, TaskAttemptState.FAILED);
|
||||
|
||||
List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
||||
jobHistoryEvents.add(EventType.TASK_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
||||
jobHistoryEvents.add(EventType.TASK_FAILED);
|
||||
recoveryChecker(recoverMapTask, TaskState.FAILED, finalAttemptStates,
|
||||
arg, jobHistoryEvents, 2L, 2L);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecoveryTaskSuccessAllAttemptsFail() {
|
||||
LOG.info("--- START: testRecoveryTaskSuccessAllAttemptsFail ---");
|
||||
|
||||
long clusterTimestamp = System.currentTimeMillis();
|
||||
EventHandler mockEventHandler = mock(EventHandler.class);
|
||||
MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
||||
mockEventHandler);
|
||||
|
||||
TaskId taskId = recoverMapTask.getID();
|
||||
JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
||||
TaskID taskID = new TaskID(jobID,
|
||||
org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
||||
|
||||
//Mock up the TaskAttempts
|
||||
Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
||||
new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
||||
|
||||
TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
||||
TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
||||
TaskAttemptState.FAILED);
|
||||
mockTaskAttempts.put(taId1, mockTAinfo1);
|
||||
|
||||
TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
||||
TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
||||
TaskAttemptState.FAILED);
|
||||
mockTaskAttempts.put(taId2, mockTAinfo2);
|
||||
|
||||
OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
||||
TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
||||
when(mockTaskInfo.getTaskStatus()).thenReturn("SUCCEEDED");
|
||||
when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
||||
when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
||||
|
||||
recoverMapTask.handle(
|
||||
new TaskRecoverEvent(taskId, mockTaskInfo, mockCommitter, true));
|
||||
|
||||
ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
||||
verify(mockEventHandler,atLeast(1)).handle(
|
||||
(org.apache.hadoop.yarn.event.Event) arg.capture());
|
||||
|
||||
Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
||||
new HashMap<TaskAttemptID, TaskAttemptState>();
|
||||
finalAttemptStates.put(taId1, TaskAttemptState.FAILED);
|
||||
finalAttemptStates.put(taId2, TaskAttemptState.FAILED);
|
||||
// check for one new attempt launched since successful attempt not found
|
||||
TaskAttemptID taId3 = new TaskAttemptID(taskID, 2000);
|
||||
finalAttemptStates.put(taId3, TaskAttemptState.NEW);
|
||||
|
||||
List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
||||
jobHistoryEvents.add(EventType.TASK_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_FAILED);
|
||||
recoveryChecker(recoverMapTask, TaskState.RUNNING, finalAttemptStates,
|
||||
arg, jobHistoryEvents, 2L, 2L);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecoveryTaskSuccessAllAttemptsSucceed() {
|
||||
LOG.info("--- START: testRecoveryTaskSuccessAllAttemptsFail ---");
|
||||
|
||||
long clusterTimestamp = System.currentTimeMillis();
|
||||
EventHandler mockEventHandler = mock(EventHandler.class);
|
||||
MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
||||
mockEventHandler);
|
||||
|
||||
TaskId taskId = recoverMapTask.getID();
|
||||
JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
||||
TaskID taskID = new TaskID(jobID,
|
||||
org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
||||
|
||||
//Mock up the TaskAttempts
|
||||
Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
||||
new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
||||
|
||||
TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
||||
TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
||||
TaskAttemptState.SUCCEEDED);
|
||||
mockTaskAttempts.put(taId1, mockTAinfo1);
|
||||
|
||||
TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
||||
TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
||||
TaskAttemptState.SUCCEEDED);
|
||||
mockTaskAttempts.put(taId2, mockTAinfo2);
|
||||
|
||||
OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
||||
TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
||||
when(mockTaskInfo.getTaskStatus()).thenReturn("SUCCEEDED");
|
||||
when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
||||
when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
||||
|
||||
recoverMapTask.handle(
|
||||
new TaskRecoverEvent(taskId, mockTaskInfo, mockCommitter, true));
|
||||
|
||||
ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
||||
verify(mockEventHandler,atLeast(1)).handle(
|
||||
(org.apache.hadoop.yarn.event.Event) arg.capture());
|
||||
|
||||
Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
||||
new HashMap<TaskAttemptID, TaskAttemptState>();
|
||||
finalAttemptStates.put(taId1, TaskAttemptState.SUCCEEDED);
|
||||
finalAttemptStates.put(taId2, TaskAttemptState.SUCCEEDED);
|
||||
|
||||
List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
||||
jobHistoryEvents.add(EventType.TASK_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_FINISHED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_FINISHED);
|
||||
jobHistoryEvents.add(EventType.TASK_FINISHED);
|
||||
recoveryChecker(recoverMapTask, TaskState.SUCCEEDED, finalAttemptStates,
|
||||
arg, jobHistoryEvents, 2L, 0L);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecoveryAllAttemptsKilled() {
|
||||
LOG.info("--- START: testRecoveryAllAttemptsKilled ---");
|
||||
|
||||
long clusterTimestamp = System.currentTimeMillis();
|
||||
EventHandler mockEventHandler = mock(EventHandler.class);
|
||||
MapTaskImpl recoverMapTask = getMockMapTask(clusterTimestamp,
|
||||
mockEventHandler);
|
||||
|
||||
TaskId taskId = recoverMapTask.getID();
|
||||
JobID jobID = new JobID(Long.toString(clusterTimestamp), 1);
|
||||
TaskID taskID = new TaskID(jobID,
|
||||
org.apache.hadoop.mapreduce.TaskType.MAP, taskId.getId());
|
||||
|
||||
//Mock up the TaskAttempts
|
||||
Map<TaskAttemptID, TaskAttemptInfo> mockTaskAttempts =
|
||||
new HashMap<TaskAttemptID, TaskAttemptInfo>();
|
||||
TaskAttemptID taId1 = new TaskAttemptID(taskID, 2);
|
||||
TaskAttemptInfo mockTAinfo1 = getMockTaskAttemptInfo(taId1,
|
||||
TaskAttemptState.KILLED);
|
||||
mockTaskAttempts.put(taId1, mockTAinfo1);
|
||||
|
||||
TaskAttemptID taId2 = new TaskAttemptID(taskID, 1);
|
||||
TaskAttemptInfo mockTAinfo2 = getMockTaskAttemptInfo(taId2,
|
||||
TaskAttemptState.KILLED);
|
||||
mockTaskAttempts.put(taId2, mockTAinfo2);
|
||||
|
||||
OutputCommitter mockCommitter = mock (OutputCommitter.class);
|
||||
TaskInfo mockTaskInfo = mock(TaskInfo.class);
|
||||
when(mockTaskInfo.getTaskStatus()).thenReturn("KILLED");
|
||||
when(mockTaskInfo.getTaskId()).thenReturn(taskID);
|
||||
when(mockTaskInfo.getAllTaskAttempts()).thenReturn(mockTaskAttempts);
|
||||
|
||||
recoverMapTask.handle(
|
||||
new TaskRecoverEvent(taskId, mockTaskInfo, mockCommitter, true));
|
||||
|
||||
ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
|
||||
verify(mockEventHandler,atLeast(1)).handle(
|
||||
(org.apache.hadoop.yarn.event.Event) arg.capture());
|
||||
|
||||
Map<TaskAttemptID, TaskAttemptState> finalAttemptStates =
|
||||
new HashMap<TaskAttemptID, TaskAttemptState>();
|
||||
finalAttemptStates.put(taId1, TaskAttemptState.KILLED);
|
||||
finalAttemptStates.put(taId2, TaskAttemptState.KILLED);
|
||||
|
||||
List<EventType> jobHistoryEvents = new ArrayList<EventType>();
|
||||
jobHistoryEvents.add(EventType.TASK_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_KILLED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_STARTED);
|
||||
jobHistoryEvents.add(EventType.MAP_ATTEMPT_KILLED);
|
||||
jobHistoryEvents.add(EventType.TASK_FAILED);
|
||||
recoveryChecker(recoverMapTask, TaskState.KILLED, finalAttemptStates,
|
||||
arg, jobHistoryEvents, 2L, 0L);
|
||||
}
|
||||
|
||||
private void recoveryChecker(MapTaskImpl checkTask, TaskState finalState,
|
||||
Map<TaskAttemptID, TaskAttemptState> finalAttemptStates,
|
||||
ArgumentCaptor<Event> arg, List<EventType> expectedJobHistoryEvents,
|
||||
long expectedMapLaunches, long expectedFailedMaps) {
|
||||
|
||||
assertEquals("Final State of Task", finalState, checkTask.getState());
|
||||
|
||||
Map<TaskAttemptId, TaskAttempt> recoveredAttempts =
|
||||
checkTask.getAttempts();
|
||||
assertEquals("Expected Number of Task Attempts",
|
||||
finalAttemptStates.size(), recoveredAttempts.size());
|
||||
for (TaskAttemptID taID : finalAttemptStates.keySet()) {
|
||||
assertEquals("Expected Task Attempt State",
|
||||
finalAttemptStates.get(taID),
|
||||
recoveredAttempts.get(TypeConverter.toYarn(taID)).getState());
|
||||
}
|
||||
|
||||
Iterator<Event> ie = arg.getAllValues().iterator();
|
||||
int eventNum = 0;
|
||||
long totalLaunchedMaps = 0;
|
||||
long totalFailedMaps = 0;
|
||||
boolean jobTaskEventReceived = false;
|
||||
|
||||
while (ie.hasNext()) {
|
||||
Object current = ie.next();
|
||||
++eventNum;
|
||||
LOG.info(eventNum + " " + current.getClass().getName());
|
||||
if (current instanceof JobHistoryEvent) {
|
||||
JobHistoryEvent jhe = (JobHistoryEvent) current;
|
||||
LOG.info(expectedJobHistoryEvents.get(0).toString() + " " +
|
||||
jhe.getHistoryEvent().getEventType().toString() + " " +
|
||||
jhe.getJobID());
|
||||
assertEquals(expectedJobHistoryEvents.get(0),
|
||||
jhe.getHistoryEvent().getEventType());
|
||||
expectedJobHistoryEvents.remove(0);
|
||||
} else if (current instanceof JobCounterUpdateEvent) {
|
||||
JobCounterUpdateEvent jcue = (JobCounterUpdateEvent) current;
|
||||
|
||||
LOG.info("JobCounterUpdateEvent "
|
||||
+ jcue.getCounterUpdates().get(0).getCounterKey()
|
||||
+ " " + jcue.getCounterUpdates().get(0).getIncrementValue());
|
||||
if (jcue.getCounterUpdates().get(0).getCounterKey() ==
|
||||
JobCounter.NUM_FAILED_MAPS) {
|
||||
totalFailedMaps += jcue.getCounterUpdates().get(0)
|
||||
.getIncrementValue();
|
||||
} else if (jcue.getCounterUpdates().get(0).getCounterKey() ==
|
||||
JobCounter.TOTAL_LAUNCHED_MAPS) {
|
||||
totalLaunchedMaps += jcue.getCounterUpdates().get(0)
|
||||
.getIncrementValue();
|
||||
}
|
||||
} else if (current instanceof JobTaskEvent) {
|
||||
JobTaskEvent jte = (JobTaskEvent) current;
|
||||
assertEquals(jte.getState(), finalState);
|
||||
jobTaskEventReceived = true;
|
||||
}
|
||||
}
|
||||
assertTrue(jobTaskEventReceived || (finalState == TaskState.RUNNING));
|
||||
assertEquals("Did not process all expected JobHistoryEvents",
|
||||
0, expectedJobHistoryEvents.size());
|
||||
assertEquals("Expected Map Launches",
|
||||
expectedMapLaunches, totalLaunchedMaps);
|
||||
assertEquals("Expected Failed Maps",
|
||||
expectedFailedMaps, totalFailedMaps);
|
||||
}
|
||||
|
||||
private MapTaskImpl getMockMapTask(long clusterTimestamp, EventHandler eh) {
|
||||
|
||||
ApplicationId appId = BuilderUtils.newApplicationId(clusterTimestamp, 1);
|
||||
JobId jobId = MRBuilderUtils.newJobId(appId, 1);
|
||||
|
||||
int partitions = 2;
|
||||
|
||||
Path remoteJobConfFile = mock(Path.class);
|
||||
JobConf conf = new JobConf();
|
||||
TaskAttemptListener taskAttemptListener = mock(TaskAttemptListener.class);
|
||||
Token<JobTokenIdentifier> jobToken =
|
||||
(Token<JobTokenIdentifier>) mock(Token.class);
|
||||
Credentials credentials = null;
|
||||
Clock clock = new SystemClock();
|
||||
int appAttemptId = 3;
|
||||
MRAppMetrics metrics = mock(MRAppMetrics.class);
|
||||
Resource minContainerRequirements = mock(Resource.class);
|
||||
when(minContainerRequirements.getMemory()).thenReturn(1000);
|
||||
|
||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||
when(clusterInfo.getMinContainerCapability()).thenReturn(
|
||||
minContainerRequirements);
|
||||
AppContext appContext = mock(AppContext.class);
|
||||
when(appContext.getClusterInfo()).thenReturn(clusterInfo);
|
||||
|
||||
TaskSplitMetaInfo taskSplitMetaInfo = mock(TaskSplitMetaInfo.class);
|
||||
MapTaskImpl mapTask = new MapTaskImpl(jobId, partitions,
|
||||
eh, remoteJobConfFile, conf,
|
||||
taskSplitMetaInfo, taskAttemptListener, jobToken, credentials, clock,
|
||||
appAttemptId, metrics, appContext);
|
||||
return mapTask;
|
||||
}
|
||||
|
||||
private TaskAttemptInfo getMockTaskAttemptInfo(TaskAttemptID tai,
|
||||
TaskAttemptState tas) {
|
||||
|
||||
ContainerId ci = mock(ContainerId.class);
|
||||
Counters counters = mock(Counters.class);
|
||||
TaskType tt = TaskType.MAP;
|
||||
|
||||
long finishTime = System.currentTimeMillis();
|
||||
|
||||
TaskAttemptInfo mockTAinfo = mock(TaskAttemptInfo.class);
|
||||
|
||||
when(mockTAinfo.getAttemptId()).thenReturn(tai);
|
||||
when(mockTAinfo.getContainerId()).thenReturn(ci);
|
||||
when(mockTAinfo.getCounters()).thenReturn(counters);
|
||||
when(mockTAinfo.getError()).thenReturn("");
|
||||
when(mockTAinfo.getFinishTime()).thenReturn(finishTime);
|
||||
when(mockTAinfo.getHostname()).thenReturn("localhost");
|
||||
when(mockTAinfo.getHttpPort()).thenReturn(23);
|
||||
when(mockTAinfo.getMapFinishTime()).thenReturn(finishTime - 1000L);
|
||||
when(mockTAinfo.getPort()).thenReturn(24);
|
||||
when(mockTAinfo.getRackname()).thenReturn("defaultRack");
|
||||
when(mockTAinfo.getShuffleFinishTime()).thenReturn(finishTime - 2000L);
|
||||
when(mockTAinfo.getShufflePort()).thenReturn(25);
|
||||
when(mockTAinfo.getSortFinishTime()).thenReturn(finishTime - 3000L);
|
||||
when(mockTAinfo.getStartTime()).thenReturn(finishTime -10000);
|
||||
when(mockTAinfo.getState()).thenReturn("task in progress");
|
||||
when(mockTAinfo.getTaskStatus()).thenReturn(tas.toString());
|
||||
when(mockTAinfo.getTaskType()).thenReturn(tt);
|
||||
when(mockTAinfo.getTrackerName()).thenReturn("TrackerName");
|
||||
return mockTAinfo;
|
||||
}
|
||||
|
||||
private void writeBadOutput(TaskAttempt attempt, Configuration conf)
|
||||
throws Exception {
|
||||
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf,
|
||||
|
@ -1145,5 +1604,16 @@ public class TestRecovery {
|
|||
public static void main(String[] arg) throws Exception {
|
||||
TestRecovery test = new TestRecovery();
|
||||
test.testCrashed();
|
||||
test.testMultipleCrashes();
|
||||
test.testOutputRecovery();
|
||||
test.testOutputRecoveryMapsOnly();
|
||||
test.testRecoveryWithOldCommiter();
|
||||
test.testSpeculative();
|
||||
test.testRecoveryWithoutShuffleSecret();
|
||||
test.testRecoverySuccessAttempt();
|
||||
test.testRecoveryAllFailAttempts();
|
||||
test.testRecoveryTaskSuccessAllAttemptsFail();
|
||||
test.testRecoveryTaskSuccessAllAttemptsSucceed();
|
||||
test.testRecoveryAllAttemptsKilled();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,9 @@ import junit.framework.TestCase;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.JobID;
|
||||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
||||
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
||||
|
@ -45,6 +47,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
|||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.yarn.YarnException;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
|
@ -86,9 +89,68 @@ import org.junit.Test;
|
|||
attemptId.setApplicationId(appId);
|
||||
JobId jobid = recordFactory.newRecordInstance(JobId.class);
|
||||
jobid.setAppId(appId);
|
||||
MRAppMaster appMaster = new TestMRApp(attemptId);
|
||||
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
|
||||
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
|
||||
JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
|
||||
appMaster.init(conf);
|
||||
appMaster.start();
|
||||
appMaster.shutDownJob();
|
||||
//test whether notifyIsLastAMRetry called
|
||||
Assert.assertEquals(true, ((TestMRApp)appMaster).getTestIsLastAMRetry());
|
||||
verify(fs).delete(stagingJobPath, true);
|
||||
}
|
||||
|
||||
@Test (timeout = 30000)
|
||||
public void testNoDeletionofStagingOnReboot() throws IOException {
|
||||
conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir);
|
||||
fs = mock(FileSystem.class);
|
||||
when(fs.delete(any(Path.class),anyBoolean())).thenReturn(true);
|
||||
String user = UserGroupInformation.getCurrentUser().getShortUserName();
|
||||
Path stagingDir = MRApps.getStagingAreaDir(conf, user);
|
||||
when(fs.exists(stagingDir)).thenReturn(true);
|
||||
ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
|
||||
ApplicationAttemptId.class);
|
||||
attemptId.setAttemptId(0);
|
||||
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
|
||||
appId.setClusterTimestamp(System.currentTimeMillis());
|
||||
appId.setId(0);
|
||||
attemptId.setApplicationId(appId);
|
||||
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
|
||||
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
|
||||
JobStateInternal.REBOOT, 4);
|
||||
appMaster.init(conf);
|
||||
appMaster.start();
|
||||
//shutdown the job, not the lastRetry
|
||||
appMaster.shutDownJob();
|
||||
//test whether notifyIsLastAMRetry called
|
||||
Assert.assertEquals(false, ((TestMRApp)appMaster).getTestIsLastAMRetry());
|
||||
verify(fs, times(0)).delete(stagingJobPath, true);
|
||||
}
|
||||
|
||||
@Test (timeout = 30000)
|
||||
public void testDeletionofStagingOnReboot() throws IOException {
|
||||
conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir);
|
||||
fs = mock(FileSystem.class);
|
||||
when(fs.delete(any(Path.class),anyBoolean())).thenReturn(true);
|
||||
String user = UserGroupInformation.getCurrentUser().getShortUserName();
|
||||
Path stagingDir = MRApps.getStagingAreaDir(conf, user);
|
||||
when(fs.exists(stagingDir)).thenReturn(true);
|
||||
ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
|
||||
ApplicationAttemptId.class);
|
||||
attemptId.setAttemptId(1);
|
||||
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
|
||||
appId.setClusterTimestamp(System.currentTimeMillis());
|
||||
appId.setId(0);
|
||||
attemptId.setApplicationId(appId);
|
||||
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
|
||||
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
|
||||
JobStateInternal.REBOOT, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
|
||||
appMaster.init(conf);
|
||||
appMaster.start();
|
||||
//shutdown the job, is lastRetry
|
||||
appMaster.shutDownJob();
|
||||
//test whether notifyIsLastAMRetry called
|
||||
Assert.assertEquals(true, ((TestMRApp)appMaster).getTestIsLastAMRetry());
|
||||
verify(fs).delete(stagingJobPath, true);
|
||||
}
|
||||
|
||||
|
@ -151,6 +213,8 @@ import org.junit.Test;
|
|||
|
||||
private class TestMRApp extends MRAppMaster {
|
||||
ContainerAllocator allocator;
|
||||
boolean testIsLastAMRetry = false;
|
||||
JobStateInternal jobStateInternal;
|
||||
|
||||
public TestMRApp(ApplicationAttemptId applicationAttemptId,
|
||||
ContainerAllocator allocator, int maxAppAttempts) {
|
||||
|
@ -160,9 +224,11 @@ import org.junit.Test;
|
|||
this.allocator = allocator;
|
||||
}
|
||||
|
||||
public TestMRApp(ApplicationAttemptId applicationAttemptId) {
|
||||
this(applicationAttemptId, null,
|
||||
MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
|
||||
public TestMRApp(ApplicationAttemptId applicationAttemptId,
|
||||
ContainerAllocator allocator, JobStateInternal jobStateInternal,
|
||||
int maxAppAttempts) {
|
||||
this(applicationAttemptId, allocator, maxAppAttempts);
|
||||
this.jobStateInternal = jobStateInternal;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -179,6 +245,31 @@ import org.junit.Test;
|
|||
return allocator;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Job createJob(Configuration conf, JobStateInternal forcedState,
|
||||
String diagnostic) {
|
||||
JobImpl jobImpl = mock(JobImpl.class);
|
||||
when(jobImpl.getInternalState()).thenReturn(this.jobStateInternal);
|
||||
JobID jobID = JobID.forName("job_1234567890000_0001");
|
||||
JobId jobId = TypeConverter.toYarn(jobID);
|
||||
when(jobImpl.getID()).thenReturn(jobId);
|
||||
((AppContext) getContext())
|
||||
.getAllJobs().put(jobImpl.getID(), jobImpl);
|
||||
return jobImpl;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
super.start();
|
||||
DefaultMetricsSystem.shutdown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void notifyIsLastAMRetry(boolean isLastAMRetry){
|
||||
testIsLastAMRetry = isLastAMRetry;
|
||||
super.notifyIsLastAMRetry(isLastAMRetry);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RMHeartbeatHandler getRMHeartbeatHandler() {
|
||||
return getStubbedHeartbeatHandler(getContext());
|
||||
|
@ -197,6 +288,9 @@ import org.junit.Test;
|
|||
protected void downloadTokensAndSetupUGI(Configuration conf) {
|
||||
}
|
||||
|
||||
public boolean getTestIsLastAMRetry(){
|
||||
return testIsLastAMRetry;
|
||||
}
|
||||
}
|
||||
|
||||
private final class MRAppTestCleanup extends MRApp {
|
||||
|
@ -222,7 +316,8 @@ import org.junit.Test;
|
|||
Job newJob = new TestJob(getJobId(), getAttemptID(), conf,
|
||||
getDispatcher().getEventHandler(),
|
||||
getTaskAttemptListener(), getContext().getClock(),
|
||||
isNewApiCommitter(), currentUser.getUserName(), getContext(),
|
||||
getCommitter(), isNewApiCommitter(),
|
||||
currentUser.getUserName(), getContext(),
|
||||
forcedState, diagnostic);
|
||||
((AppContext) getContext()).getAllJobs().put(newJob.getID(), newJob);
|
||||
|
||||
|
@ -288,7 +383,7 @@ import org.junit.Test;
|
|||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout=20000)
|
||||
public void testStagingCleanupOrder() throws Exception {
|
||||
MRAppTestCleanup app = new MRAppTestCleanup(1, 1, true,
|
||||
this.getClass().getName(), true);
|
||||
|
|
|
@ -25,6 +25,8 @@ import static org.mockito.Mockito.when;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumSet;
|
||||
import java.util.concurrent.BrokenBarrierException;
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
|
@ -35,6 +37,7 @@ import org.apache.hadoop.mapreduce.JobACL;
|
|||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.EventType;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
|
||||
import org.apache.hadoop.mapreduce.JobID;
|
||||
import org.apache.hadoop.mapreduce.JobStatus.State;
|
||||
|
@ -47,6 +50,7 @@ import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
|
|||
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
|
||||
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.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||
|
@ -57,6 +61,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobFinishEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobStartEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.InitTransition;
|
||||
|
@ -69,7 +74,6 @@ import org.apache.hadoop.yarn.SystemClock;
|
|||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.event.Event;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.state.StateMachine;
|
||||
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
||||
|
@ -114,6 +118,7 @@ public class TestJobImpl {
|
|||
conf.set(MRJobConfig.WORKFLOW_NODE_NAME, "testNodeName");
|
||||
conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key1", "value1");
|
||||
conf.set(MRJobConfig.WORKFLOW_ADJACENCY_PREFIX_STRING + "key2", "value2");
|
||||
conf.set(MRJobConfig.WORKFLOW_TAGS, "tag1,tag2");
|
||||
|
||||
|
||||
AsyncDispatcher dispatcher = new AsyncDispatcher();
|
||||
|
@ -126,12 +131,13 @@ public class TestJobImpl {
|
|||
commitHandler.start();
|
||||
|
||||
JobSubmittedEventHandler jseHandler = new JobSubmittedEventHandler("testId",
|
||||
"testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ");
|
||||
"testName", "testNodeName", "\"key2\"=\"value2\" \"key1\"=\"value1\" ",
|
||||
"tag1,tag2");
|
||||
dispatcher.register(EventType.class, jseHandler);
|
||||
JobImpl job = createStubbedJob(conf, dispatcher, 0);
|
||||
job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
|
||||
assertJobState(job, JobStateInternal.INITED);
|
||||
job.handle(new JobEvent(job.getID(), JobEventType.JOB_START));
|
||||
job.handle(new JobStartEvent(job.getID()));
|
||||
assertJobState(job, JobStateInternal.SUCCEEDED);
|
||||
dispatcher.stop();
|
||||
commitHandler.stop();
|
||||
|
@ -192,6 +198,68 @@ public class TestJobImpl {
|
|||
commitHandler.stop();
|
||||
}
|
||||
|
||||
@Test(timeout=20000)
|
||||
public void testRebootedDuringSetup() throws Exception{
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
|
||||
AsyncDispatcher dispatcher = new AsyncDispatcher();
|
||||
dispatcher.init(conf);
|
||||
dispatcher.start();
|
||||
OutputCommitter committer = new StubbedOutputCommitter() {
|
||||
@Override
|
||||
public synchronized void setupJob(JobContext jobContext)
|
||||
throws IOException {
|
||||
while(!Thread.interrupted()){
|
||||
try{
|
||||
wait();
|
||||
}catch (InterruptedException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
CommitterEventHandler commitHandler =
|
||||
createCommitterEventHandler(dispatcher, committer);
|
||||
commitHandler.init(conf);
|
||||
commitHandler.start();
|
||||
|
||||
JobImpl job = createStubbedJob(conf, dispatcher, 2);
|
||||
JobId jobId = job.getID();
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
||||
assertJobState(job, JobStateInternal.INITED);
|
||||
job.handle(new JobStartEvent(jobId));
|
||||
assertJobState(job, JobStateInternal.SETUP);
|
||||
|
||||
job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
|
||||
assertJobState(job, JobStateInternal.REBOOT);
|
||||
dispatcher.stop();
|
||||
commitHandler.stop();
|
||||
}
|
||||
|
||||
@Test(timeout=20000)
|
||||
public void testRebootedDuringCommit() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
|
||||
AsyncDispatcher dispatcher = new AsyncDispatcher();
|
||||
dispatcher.init(conf);
|
||||
dispatcher.start();
|
||||
CyclicBarrier syncBarrier = new CyclicBarrier(2);
|
||||
OutputCommitter committer = new WaitingOutputCommitter(syncBarrier, true);
|
||||
CommitterEventHandler commitHandler =
|
||||
createCommitterEventHandler(dispatcher, committer);
|
||||
commitHandler.init(conf);
|
||||
commitHandler.start();
|
||||
|
||||
JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
|
||||
completeJobTasks(job);
|
||||
assertJobState(job, JobStateInternal.COMMITTING);
|
||||
|
||||
syncBarrier.await();
|
||||
job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
|
||||
assertJobState(job, JobStateInternal.REBOOT);
|
||||
dispatcher.stop();
|
||||
commitHandler.stop();
|
||||
}
|
||||
|
||||
@Test(timeout=20000)
|
||||
public void testKilledDuringSetup() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
|
@ -220,7 +288,7 @@ public class TestJobImpl {
|
|||
JobId jobId = job.getID();
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
||||
assertJobState(job, JobStateInternal.INITED);
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_START));
|
||||
job.handle(new JobStartEvent(jobId));
|
||||
assertJobState(job, JobStateInternal.SETUP);
|
||||
|
||||
job.handle(new JobEvent(job.getID(), JobEventType.JOB_KILL));
|
||||
|
@ -287,7 +355,7 @@ public class TestJobImpl {
|
|||
JobId jobId = job.getID();
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
||||
assertJobState(job, JobStateInternal.INITED);
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_START));
|
||||
job.handle(new JobStartEvent(jobId));
|
||||
assertJobState(job, JobStateInternal.FAIL_ABORT);
|
||||
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
||||
|
@ -324,7 +392,7 @@ public class TestJobImpl {
|
|||
JobId jobId = job.getID();
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
||||
assertJobState(job, JobStateInternal.INITED);
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_START));
|
||||
job.handle(new JobStartEvent(jobId));
|
||||
assertJobState(job, JobStateInternal.SETUP);
|
||||
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
||||
|
@ -364,7 +432,7 @@ public class TestJobImpl {
|
|||
|
||||
// Verify access
|
||||
JobImpl job1 = new JobImpl(jobId, null, conf1, null, null, null, null, null,
|
||||
null, null, true, null, 0, null, null, null, null);
|
||||
null, null, null, true, null, 0, null, null, null, null);
|
||||
Assert.assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB));
|
||||
Assert.assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB));
|
||||
|
||||
|
@ -375,7 +443,7 @@ public class TestJobImpl {
|
|||
|
||||
// Verify access
|
||||
JobImpl job2 = new JobImpl(jobId, null, conf2, null, null, null, null, null,
|
||||
null, null, true, null, 0, null, null, null, null);
|
||||
null, null, null, true, null, 0, null, null, null, null);
|
||||
Assert.assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB));
|
||||
Assert.assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB));
|
||||
|
||||
|
@ -386,7 +454,7 @@ public class TestJobImpl {
|
|||
|
||||
// Verify access
|
||||
JobImpl job3 = new JobImpl(jobId, null, conf3, null, null, null, null, null,
|
||||
null, null, true, null, 0, null, null, null, null);
|
||||
null, null, null, true, null, 0, null, null, null, null);
|
||||
Assert.assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB));
|
||||
Assert.assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB));
|
||||
|
||||
|
@ -397,7 +465,7 @@ public class TestJobImpl {
|
|||
|
||||
// Verify access
|
||||
JobImpl job4 = new JobImpl(jobId, null, conf4, null, null, null, null, null,
|
||||
null, null, true, null, 0, null, null, null, null);
|
||||
null, null, null, true, null, 0, null, null, null, null);
|
||||
Assert.assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB));
|
||||
Assert.assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB));
|
||||
|
||||
|
@ -408,7 +476,7 @@ public class TestJobImpl {
|
|||
|
||||
// Verify access
|
||||
JobImpl job5 = new JobImpl(jobId, null, conf5, null, null, null, null, null,
|
||||
null, null, true, null, 0, null, null, null, null);
|
||||
null, null, null, true, null, 0, null, null, null, null);
|
||||
Assert.assertTrue(job5.checkAccess(ugi1, null));
|
||||
Assert.assertTrue(job5.checkAccess(ugi2, null));
|
||||
}
|
||||
|
@ -426,7 +494,7 @@ public class TestJobImpl {
|
|||
mock(EventHandler.class),
|
||||
null, mock(JobTokenSecretManager.class), null,
|
||||
new SystemClock(), null,
|
||||
mrAppMetrics, true, null, 0, null, null, null, null);
|
||||
mrAppMetrics, null, true, null, 0, null, null, null, null);
|
||||
job.handle(diagUpdateEvent);
|
||||
String diagnostics = job.getReport().getDiagnostics();
|
||||
Assert.assertNotNull(diagnostics);
|
||||
|
@ -437,7 +505,7 @@ public class TestJobImpl {
|
|||
mock(EventHandler.class),
|
||||
null, mock(JobTokenSecretManager.class), null,
|
||||
new SystemClock(), null,
|
||||
mrAppMetrics, true, null, 0, null, null, null, null);
|
||||
mrAppMetrics, null, true, null, 0, null, null, null, null);
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
||||
job.handle(diagUpdateEvent);
|
||||
diagnostics = job.getReport().getDiagnostics();
|
||||
|
@ -492,7 +560,7 @@ public class TestJobImpl {
|
|||
JobImpl job = new JobImpl(jobId, Records
|
||||
.newRecord(ApplicationAttemptId.class), conf, mock(EventHandler.class),
|
||||
null, new JobTokenSecretManager(), new Credentials(), null, null,
|
||||
mrAppMetrics, true, null, 0, null, null, null, null);
|
||||
mrAppMetrics, null, true, null, 0, null, null, null, null);
|
||||
InitTransition initTransition = getInitTransition(2);
|
||||
JobEvent mockJobEvent = mock(JobEvent.class);
|
||||
initTransition.transition(job, mockJobEvent);
|
||||
|
@ -533,7 +601,7 @@ public class TestJobImpl {
|
|||
JobId jobId = job.getID();
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
|
||||
assertJobState(job, JobStateInternal.INITED);
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_START));
|
||||
job.handle(new JobStartEvent(jobId));
|
||||
assertJobState(job, JobStateInternal.FAILED);
|
||||
|
||||
job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_COMPLETED));
|
||||
|
@ -597,7 +665,7 @@ public class TestJobImpl {
|
|||
StubbedJob job = createStubbedJob(conf, dispatcher, numSplits);
|
||||
job.handle(new JobEvent(job.getID(), JobEventType.JOB_INIT));
|
||||
assertJobState(job, JobStateInternal.INITED);
|
||||
job.handle(new JobEvent(job.getID(), JobEventType.JOB_START));
|
||||
job.handle(new JobStartEvent(job.getID()));
|
||||
assertJobState(job, JobStateInternal.RUNNING);
|
||||
return job;
|
||||
}
|
||||
|
@ -644,14 +712,18 @@ public class TestJobImpl {
|
|||
|
||||
private String workflowAdjacencies;
|
||||
|
||||
private String workflowTags;
|
||||
|
||||
private Boolean assertBoolean;
|
||||
|
||||
public JobSubmittedEventHandler(String workflowId, String workflowName,
|
||||
String workflowNodeName, String workflowAdjacencies) {
|
||||
String workflowNodeName, String workflowAdjacencies,
|
||||
String workflowTags) {
|
||||
this.workflowId = workflowId;
|
||||
this.workflowName = workflowName;
|
||||
this.workflowNodeName = workflowNodeName;
|
||||
this.workflowAdjacencies = workflowAdjacencies;
|
||||
this.workflowTags = workflowTags;
|
||||
assertBoolean = null;
|
||||
}
|
||||
|
||||
|
@ -673,7 +745,16 @@ public class TestJobImpl {
|
|||
setAssertValue(false);
|
||||
return;
|
||||
}
|
||||
if (!workflowAdjacencies.equals(jsEvent.getWorkflowAdjacencies())) {
|
||||
|
||||
String[] wrkflowAdj = workflowAdjacencies.split(" ");
|
||||
String[] jswrkflowAdj = jsEvent.getWorkflowAdjacencies().split(" ");
|
||||
Arrays.sort(wrkflowAdj);
|
||||
Arrays.sort(jswrkflowAdj);
|
||||
if (!Arrays.equals(wrkflowAdj, jswrkflowAdj)) {
|
||||
setAssertValue(false);
|
||||
return;
|
||||
}
|
||||
if (!workflowTags.equals(jsEvent.getWorkflowTags())) {
|
||||
setAssertValue(false);
|
||||
return;
|
||||
}
|
||||
|
@ -713,9 +794,9 @@ public class TestJobImpl {
|
|||
boolean newApiCommitter, String user, int numSplits) {
|
||||
super(jobId, applicationAttemptId, conf, eventHandler,
|
||||
null, new JobTokenSecretManager(), new Credentials(),
|
||||
new SystemClock(), null, MRAppMetrics.create(),
|
||||
newApiCommitter, user, System.currentTimeMillis(), null, null, null,
|
||||
null);
|
||||
new SystemClock(), Collections.<TaskId, TaskInfo> emptyMap(),
|
||||
MRAppMetrics.create(), null, newApiCommitter, user,
|
||||
System.currentTimeMillis(), null, null, null, null);
|
||||
|
||||
initTransition = getInitTransition(numSplits);
|
||||
localFactory = stateMachineFactory.addTransition(JobStateInternal.NEW,
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue