+ * If external modules need to access/override these methods, then
+ * they MUST be re-scoped as public/limited private.
+ */
+@Retention(RetentionPolicy.CLASS)
+@Target({ ElementType.TYPE, ElementType.METHOD, ElementType.FIELD, ElementType.CONSTRUCTOR })
+@Documented
+public @interface VisibleForTesting {
+}
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 703d5e798aafd..a4930d0d42ea5 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -133,12 +133,31 @@
com.sun.jerseyjersey-servletcompile
+
+
+ javax.enterprise
+ cdi-api
+
+
+ javax.servlet
+ servlet-api
+
+
+ ch.qos.cal10n
+ cal10n-api
+
+
-
- com.sun.jersey
+ com.github.pjfanningjersey-jsoncompile
+
+
+ org.codehaus.jettison
+ jettison
+
+ com.sun.jersey
@@ -149,6 +168,22 @@
commons-loggingcommons-loggingcompile
+
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-json-provider
+
+
+
ch.qos.reload4j
@@ -180,6 +215,15 @@
commons-lang3compile
+
+
+
+ org.codehaus.jettison
+ jettison
+ org.apache.commonscommons-text
@@ -201,8 +245,8 @@
test
- org.apache.avro
- avro
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-avro_1_11compile
@@ -278,7 +322,7 @@
org.jboss.netty
- netty
+ netty-all
@@ -314,6 +358,10 @@
org.apache.kerbykerb-simplekdc
+
+ com.fasterxml.jackson.core
+ jackson-core
+ com.fasterxml.jackson.corejackson-databind
@@ -469,6 +517,8 @@
+
+
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java
index b4a4a85674dfa..020276929af7f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java
@@ -21,7 +21,7 @@
import java.io.Closeable;
import java.io.IOException;
-import org.apache.avro.file.SeekableInput;
+import org.apache.hadoop.thirdparty.avro.file.SeekableInput;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
index c5ba3a29c2fba..a72a852084fd9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
@@ -26,7 +26,7 @@
import java.net.URISyntaxException;
import java.util.regex.Pattern;
-import org.apache.avro.reflect.Stringable;
+import org.apache.hadoop.thirdparty.avro.reflect.Stringable;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
index d3b9a21bbcff3..0e412265cd98f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
@@ -25,14 +25,14 @@
import java.util.LinkedList;
import java.util.zip.GZIPInputStream;
-import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.file.FileReader;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.avro.io.JsonEncoder;
+import org.apache.hadoop.thirdparty.avro.Schema;
+import org.apache.hadoop.thirdparty.avro.file.DataFileReader;
+import org.apache.hadoop.thirdparty.avro.file.FileReader;
+import org.apache.hadoop.thirdparty.avro.generic.GenericDatumReader;
+import org.apache.hadoop.thirdparty.avro.generic.GenericDatumWriter;
+import org.apache.hadoop.thirdparty.avro.io.DatumWriter;
+import org.apache.hadoop.thirdparty.avro.io.EncoderFactory;
+import org.apache.hadoop.thirdparty.avro.io.JsonEncoder;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
index 3ab327fe76a30..0e7446dcd241b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
@@ -33,8 +33,7 @@
import java.text.StringCharacterIterator;
import java.util.Arrays;
-import org.apache.avro.reflect.Stringable;
-
+import org.apache.hadoop.thirdparty.avro.reflect.Stringable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
index cfbc60d10452b..d18cb71dfd9b4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
@@ -21,12 +21,12 @@
import java.util.HashSet;
import java.util.Set;
-import org.apache.avro.Schema;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.reflect.ReflectData;
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.avro.reflect.ReflectDatumWriter;
+import org.apache.hadoop.thirdparty.avro.Schema;
+import org.apache.hadoop.thirdparty.avro.io.DatumReader;
+import org.apache.hadoop.thirdparty.avro.io.DatumWriter;
+import org.apache.hadoop.thirdparty.avro.reflect.ReflectData;
+import org.apache.hadoop.thirdparty.avro.reflect.ReflectDatumReader;
+import org.apache.hadoop.thirdparty.avro.reflect.ReflectDatumWriter;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
index f340cb3a98a44..cfb9593eb581d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
@@ -22,13 +22,13 @@
import java.io.InputStream;
import java.io.OutputStream;
-import org.apache.avro.Schema;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.BinaryEncoder;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.io.EncoderFactory;
+import org.apache.hadoop.thirdparty.avro.Schema;
+import org.apache.hadoop.thirdparty.avro.io.BinaryDecoder;
+import org.apache.hadoop.thirdparty.avro.io.BinaryEncoder;
+import org.apache.hadoop.thirdparty.avro.io.DatumReader;
+import org.apache.hadoop.thirdparty.avro.io.DatumWriter;
+import org.apache.hadoop.thirdparty.avro.io.DecoderFactory;
+import org.apache.hadoop.thirdparty.avro.io.EncoderFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configured;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
index e49d7a09dd53a..562c5ab96bc39 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
@@ -18,12 +18,12 @@
package org.apache.hadoop.io.serializer.avro;
-import org.apache.avro.Schema;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.avro.specific.SpecificDatumWriter;
-import org.apache.avro.specific.SpecificRecord;
+import org.apache.hadoop.thirdparty.avro.Schema;
+import org.apache.hadoop.thirdparty.avro.io.DatumReader;
+import org.apache.hadoop.thirdparty.avro.io.DatumWriter;
+import org.apache.hadoop.thirdparty.avro.specific.SpecificDatumReader;
+import org.apache.hadoop.thirdparty.avro.specific.SpecificDatumWriter;
+import org.apache.hadoop.thirdparty.avro.specific.SpecificRecord;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsJsonBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsJsonBuilder.java
index 1d62c0a29fca4..196e49e8c61b6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsJsonBuilder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsJsonBuilder.java
@@ -21,8 +21,10 @@
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.ObjectWriter;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
index 1a4e7db30415a..a12c733142b45 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
@@ -25,7 +25,7 @@
import javax.crypto.SecretKey;
-import org.apache.avro.reflect.Nullable;
+import org.apache.hadoop.thirdparty.avro.reflect.Nullable;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java
new file mode 100644
index 0000000000000..8867900d0b692
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java
@@ -0,0 +1,421 @@
+/*
+ * 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.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Static utility methods pertaining to {@link Set} instances.
+ * This class is Hadoop's internal use alternative to Guava's Sets
+ * utility class.
+ * Javadocs for majority of APIs in this class are taken from Guava's Sets
+ * class from Guava release version 27.0-jre.
+ */
+@InterfaceAudience.Private
+public final class Sets {
+
+ private static final int MAX_POWER_OF_TWO = 1 << (Integer.SIZE - 2);
+
+ private Sets() {
+ // empty
+ }
+
+ /**
+ * Creates a mutable, initially empty {@code HashSet} instance.
+ *
+ *
Note: if mutability is not required, use ImmutableSet#of()
+ * instead. If {@code E} is an {@link Enum} type, use {@link EnumSet#noneOf}
+ * instead. Otherwise, strongly consider using a {@code LinkedHashSet}
+ * instead, at the cost of increased memory footprint, to get
+ * deterministic iteration behavior.
+ *
+ * @param Generics Type E.
+ * @return a new, empty {@code TreeSet}
+ */
+ public static HashSet newHashSet() {
+ return new HashSet();
+ }
+
+ /**
+ * Creates a mutable, empty {@code TreeSet} instance sorted by the
+ * natural sort ordering of its elements.
+ *
+ *
Note: if mutability is not required, use ImmutableSortedSet#of()
+ * instead.
+ *
+ * @param Generics Type E
+ * @return a new, empty {@code TreeSet}
+ */
+ public static TreeSet newTreeSet() {
+ return new TreeSet();
+ }
+
+ /**
+ * Creates a mutable {@code HashSet} instance initially containing
+ * the given elements.
+ *
+ *
Note: if elements are non-null and won't be added or removed
+ * after this point, use ImmutableSet#of() or ImmutableSet#copyOf(Object[])
+ * instead. If {@code E} is an {@link Enum} type, use
+ * {@link EnumSet#of(Enum, Enum[])} instead. Otherwise, strongly consider
+ * using a {@code LinkedHashSet} instead, at the cost of increased memory
+ * footprint, to get deterministic iteration behavior.
+ *
+ *
This method is just a small convenience, either for
+ * {@code newHashSet(}{@link Arrays#asList}{@code (...))}, or for creating an
+ * empty set then calling {@link Collections#addAll}.
+ *
+ * @param Generics Type E.
+ * @param elements the elements that the set should contain.
+ * @return a new, empty thread-safe {@code Set}
+ */
+ @SafeVarargs
+ public static HashSet newHashSet(E... elements) {
+ HashSet set = newHashSetWithExpectedSize(elements.length);
+ Collections.addAll(set, elements);
+ return set;
+ }
+
+ /**
+ * Creates a mutable {@code HashSet} instance containing the given
+ * elements. A very thin convenience for creating an empty set then calling
+ * {@link Collection#addAll} or Iterables#addAll.
+ *
+ *
Note: if mutability is not required and the elements are
+ * non-null, use ImmutableSet#copyOf(Iterable) instead. (Or, change
+ * {@code elements} to be a FluentIterable and call {@code elements.toSet()}.)
+ *
+ *
Note: if {@code E} is an {@link Enum} type, use
+ * newEnumSet(Iterable, Class) instead.
+ *
+ * @param Generics Type E.
+ * @param elements the elements that the set should contain.
+ * @return a new, empty thread-safe {@code Set}.
+ */
+ public static HashSet newHashSet(Iterable extends E> elements) {
+ return (elements instanceof Collection)
+ ? new HashSet(cast(elements))
+ : newHashSet(elements.iterator());
+ }
+
+ /**
+ * Creates a mutable {@code TreeSet} instance containing the given
+ * elements sorted by their natural ordering.
+ *
+ *
Note: if mutability is not required, use
+ * ImmutableSortedSet#copyOf(Iterable) instead.
+ *
+ *
Note: If {@code elements} is a {@code SortedSet} with an
+ * explicit comparator, this method has different behavior than
+ * {@link TreeSet#TreeSet(SortedSet)}, which returns a {@code TreeSet}
+ * with that comparator.
+ *
+ *
Note for Java 7 and later: this method is now unnecessary and
+ * should be treated as deprecated. Instead, use the {@code TreeSet}
+ * constructor directly, taking advantage of the new
+ * "diamond" syntax.
+ *
+ *
This method is just a small convenience for creating an empty set and
+ * then calling Iterables#addAll. This method is not very useful and will
+ * likely be deprecated in the future.
+ *
+ * @param Generics Type E.
+ * @param elements the elements that the set should contain
+ * @return a new {@code TreeSet} containing those elements (minus duplicates)
+ */
+ public static TreeSet newTreeSet(
+ Iterable extends E> elements) {
+ TreeSet set = newTreeSet();
+ addAll(set, elements);
+ return set;
+ }
+
+ private static boolean addAll(TreeSet addTo,
+ Iterable extends E> elementsToAdd) {
+ if (elementsToAdd instanceof Collection) {
+ Collection extends E> c = cast(elementsToAdd);
+ return addTo.addAll(c);
+ }
+ if (elementsToAdd == null) {
+ throw new NullPointerException();
+ }
+ return addAll(addTo, elementsToAdd.iterator());
+ }
+
+ /**
+ * Creates a mutable {@code HashSet} instance containing the given
+ * elements. A very thin convenience for creating an empty set and then
+ * calling Iterators#addAll.
+ *
+ *
Note: if mutability is not required and the elements are
+ * non-null, use ImmutableSet#copyOf(Iterator) instead.
+ *
+ *
Note: if {@code E} is an {@link Enum} type, you should create
+ * an {@link EnumSet} instead.
+ *
+ *
Overall, this method is not very useful and will likely be deprecated
+ * in the future.
+ *
+ * @param Generics Type E.
+ * @param elements elements.
+ * @return a new, empty thread-safe {@code Set}.
+ */
+ public static HashSet newHashSet(Iterator extends E> elements) {
+ HashSet set = newHashSet();
+ addAll(set, elements);
+ return set;
+ }
+
+ /**
+ * Returns a new hash set using the smallest initial table size that can hold
+ * {@code expectedSize} elements without resizing. Note that this is not what
+ * {@link HashSet#HashSet(int)} does, but it is what most users want and
+ * expect it to do.
+ *
+ *
This behavior can't be broadly guaranteed, but has been tested with
+ * OpenJDK 1.7 and 1.8.
+ *
+ * @param expectedSize the number of elements you expect to add to the
+ * returned set
+ * @param Generics Type E.
+ * @return a new, empty hash set with enough capacity to hold
+ * {@code expectedSize} elements without resizing
+ * @throws IllegalArgumentException if {@code expectedSize} is negative
+ */
+ public static HashSet newHashSetWithExpectedSize(int expectedSize) {
+ return new HashSet(capacity(expectedSize));
+ }
+
+ private static Collection cast(Iterable iterable) {
+ return (Collection) iterable;
+ }
+
+ private static boolean addAll(Collection addTo,
+ Iterator extends E> iterator) {
+ if (addTo == null) {
+ throw new NullPointerException();
+ }
+ if (iterator == null) {
+ throw new NullPointerException();
+ }
+ boolean wasModified = false;
+ while (iterator.hasNext()) {
+ wasModified |= addTo.add(iterator.next());
+ }
+ return wasModified;
+ }
+
+ /**
+ * Returns the intersection of two sets as an unmodifiable set.
+ * The returned set contains all elements that are contained by both backing
+ * sets.
+ *
+ *
Results are undefined if {@code set1} and {@code set2} are sets based
+ * on different equivalence relations (as {@code HashSet}, {@code TreeSet},
+ * and the keySet of an {@code IdentityHashMap} all are).
+ *
+ * @param set1 set1.
+ * @param set2 set2.
+ * @param Generics Type E.
+ * @return a new, empty thread-safe {@code Set}.
+ */
+ public static Set intersection(final Set set1,
+ final Set set2) {
+ if (set1 == null) {
+ throw new NullPointerException("set1");
+ }
+ if (set2 == null) {
+ throw new NullPointerException("set2");
+ }
+ Set newSet = new HashSet<>(set1);
+ newSet.retainAll(set2);
+ return Collections.unmodifiableSet(newSet);
+ }
+
+ /**
+ * Returns the union of two sets as an unmodifiable set.
+ * The returned set contains all elements that are contained in either
+ * backing set.
+ *
+ *
Results are undefined if {@code set1} and {@code set2} are sets
+ * based on different equivalence relations (as {@link HashSet},
+ * {@link TreeSet}, and the {@link Map#keySet} of an
+ * {@code IdentityHashMap} all are).
+ *
+ * @param set1 set1.
+ * @param set2 set2.
+ * @param Generics Type E.
+ * @return a new, empty thread-safe {@code Set}.
+ */
+ public static Set union(
+ final Set set1, final Set set2) {
+ if (set1 == null) {
+ throw new NullPointerException("set1");
+ }
+ if (set2 == null) {
+ throw new NullPointerException("set2");
+ }
+ Set newSet = new HashSet<>(set1);
+ newSet.addAll(set2);
+ return Collections.unmodifiableSet(newSet);
+ }
+
+ /**
+ * Returns the difference of two sets as an unmodifiable set.
+ * The returned set contains all elements that are contained by {@code set1}
+ * and not contained by {@code set2}.
+ *
+ *
Results are undefined if {@code set1} and {@code set2} are sets based
+ * on different equivalence relations (as {@code HashSet}, {@code TreeSet},
+ * and the keySet of an {@code IdentityHashMap} all are).
+ *
+ * This method is used to find difference for HashSets. For TreeSets with
+ * strict order requirement, recommended method is
+ * {@link #differenceInTreeSets(Set, Set)}.
+ *
+ * @param set1 set1.
+ * @param set2 set2.
+ * @param Generics Type E.
+ * @return a new, empty thread-safe {@code Set}.
+ */
+ public static Set difference(
+ final Set set1, final Set set2) {
+ if (set1 == null) {
+ throw new NullPointerException("set1");
+ }
+ if (set2 == null) {
+ throw new NullPointerException("set2");
+ }
+ Set newSet = new HashSet<>(set1);
+ newSet.removeAll(set2);
+ return Collections.unmodifiableSet(newSet);
+ }
+
+ /**
+ * Returns the difference of two sets as an unmodifiable set.
+ * The returned set contains all elements that are contained by {@code set1}
+ * and not contained by {@code set2}.
+ *
+ *
Results are undefined if {@code set1} and {@code set2} are sets based
+ * on different equivalence relations (as {@code HashSet}, {@code TreeSet},
+ * and the keySet of an {@code IdentityHashMap} all are).
+ *
+ * This method is used to find difference for TreeSets. For HashSets,
+ * recommended method is {@link #difference(Set, Set)}.
+ *
+ * @param Generics Type E.
+ * @param set1 set1.
+ * @param set2 set2.
+ * @return a new, empty thread-safe {@code Set}.
+ */
+ public static Set differenceInTreeSets(
+ final Set set1, final Set set2) {
+ if (set1 == null) {
+ throw new NullPointerException("set1");
+ }
+ if (set2 == null) {
+ throw new NullPointerException("set2");
+ }
+ Set newSet = new TreeSet<>(set1);
+ newSet.removeAll(set2);
+ return Collections.unmodifiableSet(newSet);
+ }
+
+ /**
+ * Returns the symmetric difference of two sets as an unmodifiable set.
+ * The returned set contains all elements that are contained in either
+ * {@code set1} or {@code set2} but not in both. The iteration order of the
+ * returned set is undefined.
+ *
+ *
Results are undefined if {@code set1} and {@code set2} are sets based
+ * on different equivalence relations (as {@code HashSet}, {@code TreeSet},
+ * and the keySet of an {@code IdentityHashMap} all are).
+ *
+ * @param set1 set1.
+ * @param set2 set2.
+ * @param Generics Type E.
+ * @return a new, empty thread-safe {@code Set}.
+ */
+ public static Set symmetricDifference(
+ final Set set1, final Set set2) {
+ if (set1 == null) {
+ throw new NullPointerException("set1");
+ }
+ if (set2 == null) {
+ throw new NullPointerException("set2");
+ }
+ Set intersection = new HashSet<>(set1);
+ intersection.retainAll(set2);
+ Set symmetricDifference = new HashSet<>(set1);
+ symmetricDifference.addAll(set2);
+ symmetricDifference.removeAll(intersection);
+ return Collections.unmodifiableSet(symmetricDifference);
+ }
+
+ /**
+ * Creates a thread-safe set backed by a hash map. The set is backed by a
+ * {@link ConcurrentHashMap} instance, and thus carries the same concurrency
+ * guarantees.
+ *
+ *
Unlike {@code HashSet}, this class does NOT allow {@code null} to be
+ * used as an element. The set is serializable.
+ *
+ * @param Generics Type.
+ * @return a new, empty thread-safe {@code Set}
+ */
+ public static Set newConcurrentHashSet() {
+ return Collections.newSetFromMap(new ConcurrentHashMap());
+ }
+
+ /**
+ * Returns a capacity that is sufficient to keep the map from being resized
+ * as long as it grows no larger than expectedSize and the load factor
+ * is ≥ its default (0.75).
+ * The implementation of this method is adapted from Guava version 27.0-jre.
+ */
+ private static int capacity(int expectedSize) {
+ if (expectedSize < 3) {
+ if (expectedSize < 0) {
+ throw new IllegalArgumentException(
+ "expectedSize cannot be negative but was: " + expectedSize);
+ }
+ return expectedSize + 1;
+ }
+ if (expectedSize < MAX_POWER_OF_TWO) {
+ // This is the calculation used in JDK8 to resize when a putAll
+ // happens; it seems to be the most conservative calculation we
+ // can make. 0.75 is the default load factor.
+ return (int) ((float) expectedSize / 0.75F + 1.0F);
+ }
+ return Integer.MAX_VALUE; // any large value
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/AvroTestUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/AvroTestUtil.java
index ec76ea008077d..d01e0a94873c4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/AvroTestUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/AvroTestUtil.java
@@ -21,12 +21,12 @@
import java.io.ByteArrayOutputStream;
import java.lang.reflect.Type;
-import org.apache.avro.Schema;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.avro.reflect.ReflectData;
-import org.apache.avro.reflect.ReflectDatumWriter;
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.avro.io.DecoderFactory;
+import org.apache.hadoop.thirdparty.avro.Schema;
+import org.apache.hadoop.thirdparty.avro.io.EncoderFactory;
+import org.apache.hadoop.thirdparty.avro.reflect.ReflectData;
+import org.apache.hadoop.thirdparty.avro.reflect.ReflectDatumWriter;
+import org.apache.hadoop.thirdparty.avro.reflect.ReflectDatumReader;
+import org.apache.hadoop.thirdparty.avro.io.DecoderFactory;
import static org.junit.Assert.assertEquals;
@@ -41,7 +41,7 @@ public static void testReflect(Object value, Type type, String schema)
// check that schema matches expected
Schema s = ReflectData.get().getSchema(type);
- assertEquals(Schema.parse(schema), s);
+ assertEquals(new Schema.Parser().parse(schema), s);
// check that value is serialized correctly
ReflectDatumWriter
io.netty
- netty
+ netty-allcompilecom.google.guavaguava
+
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-guava
+
+
+ org.assertj
+ assertj-core
+ test
+
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
index 0ff3084bf3eb9..58d3e51f2bdfb 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
@@ -41,6 +41,8 @@ abstract public class MountdBase {
private final RpcProgram rpcProgram;
private int udpBoundPort; // Will set after server starts
private int tcpBoundPort; // Will set after server starts
+ private SimpleUdpServer udpServer = null;
+ private SimpleTcpServer tcpServer = null;
public RpcProgram getRpcProgram() {
return rpcProgram;
@@ -57,7 +59,7 @@ public MountdBase(RpcProgram program) throws IOException {
/* Start UDP server */
private void startUDPServer() {
- SimpleUdpServer udpServer = new SimpleUdpServer(rpcProgram.getPort(),
+ udpServer = new SimpleUdpServer(rpcProgram.getPort(),
rpcProgram, 1);
rpcProgram.startDaemons();
try {
@@ -76,7 +78,7 @@ private void startUDPServer() {
/* Start TCP server */
private void startTCPServer() {
- SimpleTcpServer tcpServer = new SimpleTcpServer(rpcProgram.getPort(),
+ tcpServer = new SimpleTcpServer(rpcProgram.getPort(),
rpcProgram, 1);
rpcProgram.startDaemons();
try {
@@ -118,6 +120,14 @@ public void stop() {
rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, tcpBoundPort);
tcpBoundPort = 0;
}
+ if (udpServer != null) {
+ udpServer.shutdown();
+ udpServer = null;
+ }
+ if (tcpServer != null) {
+ tcpServer.shutdown();
+ tcpServer = null;
+ }
}
/**
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
index ff83a5f19bee1..e6ea29b42bff4 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
@@ -35,6 +35,7 @@ public abstract class Nfs3Base {
public static final Logger LOG = LoggerFactory.getLogger(Nfs3Base.class);
private final RpcProgram rpcProgram;
private int nfsBoundPort; // Will set after server starts
+ private SimpleTcpServer tcpServer = null;
public RpcProgram getRpcProgram() {
return rpcProgram;
@@ -61,7 +62,7 @@ public void start(boolean register) {
}
private void startTCPServer() {
- SimpleTcpServer tcpServer = new SimpleTcpServer(rpcProgram.getPort(),
+ tcpServer = new SimpleTcpServer(rpcProgram.getPort(),
rpcProgram, 0);
rpcProgram.startDaemons();
try {
@@ -84,6 +85,10 @@ public void stop() {
nfsBoundPort = 0;
}
rpcProgram.stopDaemons();
+ if (tcpServer != null) {
+ tcpServer.shutdown();
+ tcpServer = null;
+ }
}
/**
* Priority of the nfsd shutdown hook.
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RegistrationClient.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RegistrationClient.java
index c8528ba4d558f..c96f1d53bb4c5 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RegistrationClient.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RegistrationClient.java
@@ -19,10 +19,9 @@
import java.util.Arrays;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.oncrpc.RpcAcceptedReply.AcceptState;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.MessageEvent;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -58,10 +57,10 @@ private boolean validMessageLength(int len) {
}
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
- ChannelBuffer buf = (ChannelBuffer) e.getMessage(); // Read reply
+ public void channelRead(ChannelHandlerContext ctx, Object msg) {
+ ByteBuf buf = (ByteBuf) msg; // Read reply
if (!validMessageLength(buf.readableBytes())) {
- e.getChannel().close();
+ ctx.channel().close();
return;
}
@@ -83,7 +82,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
RpcDeniedReply deniedReply = (RpcDeniedReply) reply;
handle(deniedReply);
}
- e.getChannel().close(); // shutdown now that request is complete
+ ctx.channel().close(); // shutdown now that request is complete
}
private void handle(RpcDeniedReply deniedReply) {
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcInfo.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcInfo.java
index b434d79285c6f..aba8e9ea2624e 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcInfo.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcInfo.java
@@ -19,9 +19,9 @@
import java.net.SocketAddress;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
/**
* RpcInfo records all contextual information of an RPC message. It contains
@@ -29,11 +29,11 @@
*/
public final class RpcInfo {
private final RpcMessage header;
- private final ChannelBuffer data;
+ private final ByteBuf data;
private final Channel channel;
private final SocketAddress remoteAddress;
- public RpcInfo(RpcMessage header, ChannelBuffer data,
+ public RpcInfo(RpcMessage header, ByteBuf data,
ChannelHandlerContext channelContext, Channel channel,
SocketAddress remoteAddress) {
this.header = header;
@@ -46,7 +46,7 @@ public RpcMessage header() {
return header;
}
- public ChannelBuffer data() {
+ public ByteBuf data() {
return data;
}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java
index 194336ab1aa9e..4d5323c57643f 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java
@@ -22,16 +22,17 @@
import java.net.InetSocketAddress;
import java.net.SocketAddress;
-import com.google.common.annotations.VisibleForTesting;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import io.netty.util.ReferenceCountUtil;
import org.apache.hadoop.oncrpc.RpcAcceptedReply.AcceptState;
+import org.apache.hadoop.oncrpc.security.Verifier;
import org.apache.hadoop.oncrpc.security.VerifierNone;
import org.apache.hadoop.portmap.PortmapMapping;
import org.apache.hadoop.portmap.PortmapRequest;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -39,7 +40,7 @@
* Class for writing RPC server programs based on RFC 1050. Extend this class
* and implement {@link #handleInternal} to handle the requests received.
*/
-public abstract class RpcProgram extends SimpleChannelUpstreamHandler {
+public abstract class RpcProgram extends ChannelInboundHandlerAdapter {
static final Logger LOG = LoggerFactory.getLogger(RpcProgram.class);
public static final int RPCB_PORT = 111;
private final String program;
@@ -161,9 +162,18 @@ public void startDaemons() {}
public void stopDaemons() {}
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+ public void channelRead(ChannelHandlerContext ctx, Object msg)
+ throws Exception {
+ RpcInfo info = (RpcInfo) msg;
+ try {
+ channelRead(ctx, info);
+ } finally {
+ ReferenceCountUtil.release(info.data());
+ }
+ }
+
+ private void channelRead(ChannelHandlerContext ctx, RpcInfo info)
throws Exception {
- RpcInfo info = (RpcInfo) e.getMessage();
RpcCall call = (RpcCall) info.header();
SocketAddress remoteAddress = info.remoteAddress();
@@ -221,7 +231,7 @@ private void sendAcceptedReply(RpcCall call, SocketAddress remoteAddress,
out.writeInt(lowProgVersion);
out.writeInt(highProgVersion);
}
- ChannelBuffer b = ChannelBuffers.wrappedBuffer(out.asReadOnlyWrap()
+ ByteBuf b = Unpooled.wrappedBuffer(out.asReadOnlyWrap()
.buffer());
RpcResponse rsp = new RpcResponse(b, remoteAddress);
RpcUtil.sendRpcResponse(ctx, rsp);
@@ -234,7 +244,7 @@ protected static void sendRejectedReply(RpcCall call,
RpcReply.ReplyState.MSG_DENIED,
RpcDeniedReply.RejectState.AUTH_ERROR, new VerifierNone());
reply.write(out);
- ChannelBuffer buf = ChannelBuffers.wrappedBuffer(out.asReadOnlyWrap()
+ ByteBuf buf = Unpooled.wrappedBuffer(out.asReadOnlyWrap()
.buffer());
RpcResponse rsp = new RpcResponse(buf, remoteAddress);
RpcUtil.sendRpcResponse(ctx, rsp);
@@ -257,4 +267,4 @@ public int getPort() {
public int getPortmapUdpTimeoutMillis() {
return portmapUdpTimeoutMillis;
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcResponse.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcResponse.java
index 2e45e6100b108..0d6431f68bd5a 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcResponse.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcResponse.java
@@ -19,27 +19,30 @@
import java.net.SocketAddress;
-import org.jboss.netty.buffer.ChannelBuffer;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.DefaultAddressedEnvelope;
/**
* RpcResponse encapsulates a response to a RPC request. It contains the data
* that is going to cross the wire, as well as the information of the remote
* peer.
*/
-public class RpcResponse {
- private final ChannelBuffer data;
- private final SocketAddress remoteAddress;
+public class RpcResponse extends
+ DefaultAddressedEnvelope {
+ public RpcResponse(ByteBuf message, SocketAddress recipient) {
+ super(message, recipient, null);
+ }
- public RpcResponse(ChannelBuffer data, SocketAddress remoteAddress) {
- this.data = data;
- this.remoteAddress = remoteAddress;
+ public RpcResponse(ByteBuf message, SocketAddress recipient,
+ SocketAddress sender) {
+ super(message, recipient, sender);
}
- public ChannelBuffer data() {
- return data;
+ public ByteBuf data() {
+ return this.content();
}
public SocketAddress remoteAddress() {
- return remoteAddress;
+ return this.recipient();
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java
index cebebd27d0c4b..6fda4b1bd5e8d 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcUtil.java
@@ -17,16 +17,18 @@
*/
package org.apache.hadoop.oncrpc;
+import java.net.SocketAddress;
import java.nio.ByteBuffer;
-
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.handler.codec.frame.FrameDecoder;
+import java.util.List;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.socket.DatagramPacket;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -43,16 +45,16 @@ public static int getNewXid(String caller) {
public static void sendRpcResponse(ChannelHandlerContext ctx,
RpcResponse response) {
- Channels.fireMessageReceived(ctx, response);
+ ctx.fireChannelRead(response);
}
- public static FrameDecoder constructRpcFrameDecoder() {
+ public static ByteToMessageDecoder constructRpcFrameDecoder() {
return new RpcFrameDecoder();
}
- public static final SimpleChannelUpstreamHandler STAGE_RPC_MESSAGE_PARSER = new RpcMessageParserStage();
- public static final SimpleChannelUpstreamHandler STAGE_RPC_TCP_RESPONSE = new RpcTcpResponseStage();
- public static final SimpleChannelUpstreamHandler STAGE_RPC_UDP_RESPONSE = new RpcUdpResponseStage();
+ public static final ChannelInboundHandlerAdapter STAGE_RPC_MESSAGE_PARSER = new RpcMessageParserStage();
+ public static final ChannelInboundHandlerAdapter STAGE_RPC_TCP_RESPONSE = new RpcTcpResponseStage();
+ public static final ChannelInboundHandlerAdapter STAGE_RPC_UDP_RESPONSE = new RpcUdpResponseStage();
/**
* An RPC client can separate a RPC message into several frames (i.e.,
@@ -62,44 +64,39 @@ public static FrameDecoder constructRpcFrameDecoder() {
* RpcFrameDecoder is a stateful pipeline stage. It has to be constructed for
* each RPC client.
*/
- static class RpcFrameDecoder extends FrameDecoder {
+ static class RpcFrameDecoder extends ByteToMessageDecoder {
public static final Logger LOG =
LoggerFactory.getLogger(RpcFrameDecoder.class);
- private ChannelBuffer currentFrame;
+ private volatile boolean isLast;
@Override
- protected Object decode(ChannelHandlerContext ctx, Channel channel,
- ChannelBuffer buf) {
+ protected void decode(ChannelHandlerContext ctx, ByteBuf buf,
+ List out) {
- if (buf.readableBytes() < 4)
- return null;
+ if (buf.readableBytes() < 4) {
+ return;
+ }
buf.markReaderIndex();
byte[] fragmentHeader = new byte[4];
buf.readBytes(fragmentHeader);
int length = XDR.fragmentSize(fragmentHeader);
- boolean isLast = XDR.isLastFragment(fragmentHeader);
+ isLast = XDR.isLastFragment(fragmentHeader);
if (buf.readableBytes() < length) {
buf.resetReaderIndex();
- return null;
+ return;
}
- ChannelBuffer newFragment = buf.readSlice(length);
- if (currentFrame == null) {
- currentFrame = newFragment;
- } else {
- currentFrame = ChannelBuffers.wrappedBuffer(currentFrame, newFragment);
- }
+ ByteBuf newFragment = buf.readSlice(length);
+ newFragment.retain();
+ out.add(newFragment);
+ }
- if (isLast) {
- ChannelBuffer completeFrame = currentFrame;
- currentFrame = null;
- return completeFrame;
- } else {
- return null;
- }
+ @VisibleForTesting
+ public boolean isLast() {
+ return isLast;
}
}
@@ -107,30 +104,45 @@ protected Object decode(ChannelHandlerContext ctx, Channel channel,
* RpcMessageParserStage parses the network bytes and encapsulates the RPC
* request into a RpcInfo instance.
*/
- static final class RpcMessageParserStage extends SimpleChannelUpstreamHandler {
+ @ChannelHandler.Sharable
+ static final class RpcMessageParserStage extends ChannelInboundHandlerAdapter {
private static final Logger LOG = LoggerFactory
.getLogger(RpcMessageParserStage.class);
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+ public void channelRead(ChannelHandlerContext ctx, Object msg)
throws Exception {
- ChannelBuffer buf = (ChannelBuffer) e.getMessage();
- ByteBuffer b = buf.toByteBuffer().asReadOnlyBuffer();
+ ByteBuf buf;
+ SocketAddress remoteAddress;
+ if (msg instanceof DatagramPacket) {
+ DatagramPacket packet = (DatagramPacket)msg;
+ buf = packet.content();
+ remoteAddress = packet.sender();
+ } else {
+ buf = (ByteBuf) msg;
+ remoteAddress = ctx.channel().remoteAddress();
+ }
+
+ ByteBuffer b = buf.nioBuffer().asReadOnlyBuffer();
XDR in = new XDR(b, XDR.State.READING);
RpcInfo info = null;
try {
RpcCall callHeader = RpcCall.read(in);
- ChannelBuffer dataBuffer = ChannelBuffers.wrappedBuffer(in.buffer()
- .slice());
- info = new RpcInfo(callHeader, dataBuffer, ctx, e.getChannel(),
- e.getRemoteAddress());
+ ByteBuf dataBuffer = buf.slice(b.position(), b.remaining());
+ info = new RpcInfo(callHeader, dataBuffer, ctx, ctx.channel(),
+ remoteAddress);
} catch (Exception exc) {
- LOG.info("Malformed RPC request from " + e.getRemoteAddress());
+ LOG.info("Malformed RPC request from " + remoteAddress);
+ } finally {
+ // only release buffer if it is not passed to downstream handler
+ if (info == null) {
+ buf.release();
+ }
}
if (info != null) {
- Channels.fireMessageReceived(ctx, info);
+ ctx.fireChannelRead(info);
}
}
}
@@ -139,16 +151,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
* RpcTcpResponseStage sends an RpcResponse across the wire with the
* appropriate fragment header.
*/
- private static class RpcTcpResponseStage extends SimpleChannelUpstreamHandler {
+ @ChannelHandler.Sharable
+ private static class RpcTcpResponseStage extends ChannelInboundHandlerAdapter {
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+ public void channelRead(ChannelHandlerContext ctx, Object msg)
throws Exception {
- RpcResponse r = (RpcResponse) e.getMessage();
+ RpcResponse r = (RpcResponse) msg;
byte[] fragmentHeader = XDR.recordMark(r.data().readableBytes(), true);
- ChannelBuffer header = ChannelBuffers.wrappedBuffer(fragmentHeader);
- ChannelBuffer d = ChannelBuffers.wrappedBuffer(header, r.data());
- e.getChannel().write(d);
+ ByteBuf header = Unpooled.wrappedBuffer(fragmentHeader);
+ ByteBuf d = Unpooled.wrappedBuffer(header, r.data());
+ ctx.channel().writeAndFlush(d);
}
}
@@ -156,14 +169,17 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
* RpcUdpResponseStage sends an RpcResponse as a UDP packet, which does not
* require a fragment header.
*/
+ @ChannelHandler.Sharable
private static final class RpcUdpResponseStage extends
- SimpleChannelUpstreamHandler {
+ ChannelInboundHandlerAdapter {
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+ public void channelRead(ChannelHandlerContext ctx, Object msg)
throws Exception {
- RpcResponse r = (RpcResponse) e.getMessage();
- e.getChannel().write(r.data(), r.remoteAddress());
+ RpcResponse r = (RpcResponse) msg;
+ // TODO: check out https://github.com/netty/netty/issues/1282 for
+ // correct usage
+ ctx.channel().writeAndFlush(r.data());
}
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java
index 32e1b4b839218..7cfef6439b059 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClient.java
@@ -18,15 +18,16 @@
package org.apache.hadoop.oncrpc;
import java.net.InetSocketAddress;
-import java.util.concurrent.Executors;
-import org.jboss.netty.bootstrap.ClientBootstrap;
-import org.jboss.netty.channel.ChannelFactory;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* A simple TCP based RPC client which just sends a request to a server.
@@ -35,8 +36,9 @@ public class SimpleTcpClient {
protected final String host;
protected final int port;
protected final XDR request;
- protected ChannelPipelineFactory pipelineFactory;
protected final boolean oneShot;
+ private NioEventLoopGroup workerGroup;
+ private ChannelFuture future;
public SimpleTcpClient(String host, int port, XDR request) {
this(host,port, request, true);
@@ -48,40 +50,54 @@ public SimpleTcpClient(String host, int port, XDR request, Boolean oneShot) {
this.request = request;
this.oneShot = oneShot;
}
-
- protected ChannelPipelineFactory setPipelineFactory() {
- this.pipelineFactory = new ChannelPipelineFactory() {
+
+ protected ChannelInitializer setChannelHandler() {
+ return new ChannelInitializer() {
@Override
- public ChannelPipeline getPipeline() {
- return Channels.pipeline(
+ protected void initChannel(SocketChannel ch) throws Exception {
+ ChannelPipeline p = ch.pipeline();
+ p.addLast(
RpcUtil.constructRpcFrameDecoder(),
- new SimpleTcpClientHandler(request));
+ new SimpleTcpClientHandler(request)
+ );
}
};
- return this.pipelineFactory;
}
+ @VisibleForTesting
public void run() {
// Configure the client.
- ChannelFactory factory = new NioClientSocketChannelFactory(
- Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), 1, 1);
- ClientBootstrap bootstrap = new ClientBootstrap(factory);
-
- // Set up the pipeline factory.
- bootstrap.setPipelineFactory(setPipelineFactory());
-
- bootstrap.setOption("tcpNoDelay", true);
- bootstrap.setOption("keepAlive", true);
+ workerGroup = new NioEventLoopGroup();
+ Bootstrap bootstrap = new Bootstrap()
+ .group(workerGroup)
+ .channel(NioSocketChannel.class);
- // Start the connection attempt.
- ChannelFuture future = bootstrap.connect(new InetSocketAddress(host, port));
+ try {
+ future = bootstrap.handler(setChannelHandler())
+ .option(ChannelOption.TCP_NODELAY, true)
+ .option(ChannelOption.SO_KEEPALIVE, true)
+ .connect(new InetSocketAddress(host, port)).sync();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ } finally {
+ if (oneShot) {
+ stop();
+ }
+ }
+ }
- if (oneShot) {
- // Wait until the connection is closed or the connection attempt fails.
- future.getChannel().getCloseFuture().awaitUninterruptibly();
+ public void stop() {
+ try {
+ if (future != null) {
+ // Wait until the connection is closed or the connection attempt fails.
+ future.channel().closeFuture().sync();
+ }
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ } finally {
// Shut down thread pools to exit.
- bootstrap.releaseExternalResources();
+ workerGroup.shutdownGracefully();
}
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java
index 23b6682361c9b..1acefc857f830 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java
@@ -17,19 +17,19 @@
*/
package org.apache.hadoop.oncrpc;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelStateEvent;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelHandler;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.util.ReferenceCountUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A simple TCP based RPC client handler used by {@link SimpleTcpServer}.
*/
-public class SimpleTcpClientHandler extends SimpleChannelHandler {
+public class SimpleTcpClientHandler extends ChannelInboundHandlerAdapter {
public static final Logger LOG =
LoggerFactory.getLogger(SimpleTcpClient.class);
protected final XDR request;
@@ -39,13 +39,13 @@ public SimpleTcpClientHandler(XDR request) {
}
@Override
- public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) {
+ public void channelActive(ChannelHandlerContext ctx) throws Exception {
// Send the request
if (LOG.isDebugEnabled()) {
LOG.debug("sending PRC request");
}
- ChannelBuffer outBuf = XDR.writeMessageTcp(request, true);
- e.getChannel().write(outBuf);
+ ByteBuf outBuf = XDR.writeMessageTcp(request, true);
+ ctx.channel().writeAndFlush(outBuf);
}
/**
@@ -53,13 +53,13 @@ public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) {
* more interaction with the server.
*/
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
- e.getChannel().close();
+ public void channelRead(ChannelHandlerContext ctx, Object msg) {
+ ctx.channel().close();
}
@Override
- public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) {
- LOG.warn("Unexpected exception from downstream: ", e.getCause());
- e.getChannel().close();
+ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+ LOG.warn("Unexpected exception from downstream: ", cause.getCause());
+ ctx.channel().close();
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java
index 177fa3d80b1b4..29155c80b1846 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpServer.java
@@ -20,14 +20,17 @@
import java.net.InetSocketAddress;
import java.util.concurrent.Executors;
-import org.jboss.netty.bootstrap.ServerBootstrap;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelFactory;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -39,9 +42,11 @@ public class SimpleTcpServer {
LoggerFactory.getLogger(SimpleTcpServer.class);
protected final int port;
protected int boundPort = -1; // Will be set after server starts
- protected final SimpleChannelUpstreamHandler rpcProgram;
+ protected final ChannelInboundHandlerAdapter rpcProgram;
private ServerBootstrap server;
private Channel ch;
+ private EventLoopGroup bossGroup;
+ private EventLoopGroup workerGroup;
/** The maximum number of I/O worker threads */
protected final int workerCount;
@@ -57,37 +62,32 @@ public SimpleTcpServer(int port, RpcProgram program, int workercount) {
this.workerCount = workercount;
}
- public void run() {
+ public void run() throws InterruptedException {
// Configure the Server.
- ChannelFactory factory;
- if (workerCount == 0) {
- // Use default workers: 2 * the number of available processors
- factory = new NioServerSocketChannelFactory(
- Executors.newCachedThreadPool(), Executors.newCachedThreadPool());
- } else {
- factory = new NioServerSocketChannelFactory(
- Executors.newCachedThreadPool(), Executors.newCachedThreadPool(),
- workerCount);
- }
+ bossGroup = new NioEventLoopGroup();
+ workerGroup = new NioEventLoopGroup(workerCount, Executors.newCachedThreadPool());
- server = new ServerBootstrap(factory);
- server.setPipelineFactory(new ChannelPipelineFactory() {
+ server = new ServerBootstrap();
+ server.group(bossGroup, workerGroup)
+ .channel(NioServerSocketChannel.class)
+ .childHandler(new ChannelInitializer() {
@Override
- public ChannelPipeline getPipeline() throws Exception {
- return Channels.pipeline(RpcUtil.constructRpcFrameDecoder(),
+ protected void initChannel(SocketChannel ch) throws Exception {
+ ChannelPipeline p = ch.pipeline();
+ p.addLast(RpcUtil.constructRpcFrameDecoder(),
RpcUtil.STAGE_RPC_MESSAGE_PARSER, rpcProgram,
RpcUtil.STAGE_RPC_TCP_RESPONSE);
- }
- });
- server.setOption("child.tcpNoDelay", true);
- server.setOption("child.keepAlive", true);
- server.setOption("child.reuseAddress", true);
- server.setOption("reuseAddress", true);
+ }})
+ .childOption(ChannelOption.TCP_NODELAY, true)
+ .childOption(ChannelOption.SO_KEEPALIVE, true)
+ .childOption(ChannelOption.SO_REUSEADDR, true)
+ .option(ChannelOption.SO_REUSEADDR, true);
// Listen to TCP port
- ch = server.bind(new InetSocketAddress(port));
- InetSocketAddress socketAddr = (InetSocketAddress) ch.getLocalAddress();
+ ChannelFuture f = server.bind(new InetSocketAddress(port)).sync();
+ ch = f.channel();
+ InetSocketAddress socketAddr = (InetSocketAddress) ch.localAddress();
boundPort = socketAddr.getPort();
LOG.info("Started listening to TCP requests at port " + boundPort + " for "
@@ -102,9 +102,17 @@ public int getBoundPort() {
public void shutdown() {
if (ch != null) {
ch.close().awaitUninterruptibly();
+ ch = null;
+ }
+
+ if (workerGroup != null) {
+ workerGroup.shutdownGracefully();
+ workerGroup = null;
}
- if (server != null) {
- server.releaseExternalResources();
+
+ if (bossGroup != null) {
+ bossGroup.shutdownGracefully();
+ bossGroup = null;
}
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleUdpServer.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleUdpServer.java
index e65003ca64beb..516503c323a08 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleUdpServer.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleUdpServer.java
@@ -20,12 +20,16 @@
import java.net.InetSocketAddress;
import java.util.concurrent.Executors;
-import org.jboss.netty.bootstrap.ConnectionlessBootstrap;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.channel.socket.DatagramChannelFactory;
-import org.jboss.netty.channel.socket.nio.NioDatagramChannelFactory;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioDatagramChannel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -39,36 +43,45 @@ public class SimpleUdpServer {
private final int RECEIVE_BUFFER_SIZE = 65536;
protected final int port;
- protected final SimpleChannelUpstreamHandler rpcProgram;
+ protected final ChannelInboundHandlerAdapter rpcProgram;
protected final int workerCount;
protected int boundPort = -1; // Will be set after server starts
- private ConnectionlessBootstrap server;
+ private Bootstrap server;
private Channel ch;
+ private EventLoopGroup workerGroup;
- public SimpleUdpServer(int port, SimpleChannelUpstreamHandler program,
+ public SimpleUdpServer(int port, ChannelInboundHandlerAdapter program,
int workerCount) {
this.port = port;
this.rpcProgram = program;
this.workerCount = workerCount;
}
- public void run() {
- // Configure the client.
- DatagramChannelFactory f = new NioDatagramChannelFactory(
- Executors.newCachedThreadPool(), workerCount);
+ public void run() throws InterruptedException {
+ workerGroup = new NioEventLoopGroup(workerCount, Executors.newCachedThreadPool());
- server = new ConnectionlessBootstrap(f);
- server.setPipeline(Channels.pipeline(RpcUtil.STAGE_RPC_MESSAGE_PARSER,
- rpcProgram, RpcUtil.STAGE_RPC_UDP_RESPONSE));
-
- server.setOption("broadcast", "false");
- server.setOption("sendBufferSize", SEND_BUFFER_SIZE);
- server.setOption("receiveBufferSize", RECEIVE_BUFFER_SIZE);
- server.setOption("reuseAddress", true);
+ server = new Bootstrap();
+ server.group(workerGroup)
+ .channel(NioDatagramChannel.class)
+ .option(ChannelOption.SO_BROADCAST, true)
+ .option(ChannelOption.SO_SNDBUF, SEND_BUFFER_SIZE)
+ .option(ChannelOption.SO_RCVBUF, RECEIVE_BUFFER_SIZE)
+ .option(ChannelOption.SO_REUSEADDR, true)
+ .handler(new ChannelInitializer() {
+ @Override protected void initChannel(NioDatagramChannel ch)
+ throws Exception {
+ ChannelPipeline p = ch.pipeline();
+ p.addLast(
+ RpcUtil.STAGE_RPC_MESSAGE_PARSER,
+ rpcProgram,
+ RpcUtil.STAGE_RPC_UDP_RESPONSE);
+ }
+ });
// Listen to the UDP port
- ch = server.bind(new InetSocketAddress(port));
- InetSocketAddress socketAddr = (InetSocketAddress) ch.getLocalAddress();
+ ChannelFuture f = server.bind(new InetSocketAddress(port)).sync();
+ ch = f.channel();
+ InetSocketAddress socketAddr = (InetSocketAddress) ch.localAddress();
boundPort = socketAddr.getPort();
LOG.info("Started listening to UDP requests at port " + boundPort + " for "
@@ -83,9 +96,11 @@ public int getBoundPort() {
public void shutdown() {
if (ch != null) {
ch.close().awaitUninterruptibly();
+ ch = null;
}
- if (server != null) {
- server.releaseExternalResources();
+ if (workerGroup != null) {
+ workerGroup.shutdownGracefully();
+ workerGroup = null;
}
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
index acdc51c954521..af9999d3cac43 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/XDR.java
@@ -20,8 +20,8 @@
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@@ -242,7 +242,7 @@ static byte[] recordMark(int size, boolean last) {
* @param last specifies last request or not
* @return TCP buffer
*/
- public static ChannelBuffer writeMessageTcp(XDR request, boolean last) {
+ public static ByteBuf writeMessageTcp(XDR request, boolean last) {
Preconditions.checkState(request.state == XDR.State.WRITING);
ByteBuffer b = request.buf.duplicate();
b.flip();
@@ -250,7 +250,7 @@ public static ChannelBuffer writeMessageTcp(XDR request, boolean last) {
ByteBuffer headerBuf = ByteBuffer.wrap(fragmentHeader);
// TODO: Investigate whether making a copy of the buffer is necessary.
- return ChannelBuffers.copiedBuffer(headerBuf, b);
+ return Unpooled.wrappedBuffer(headerBuf, b);
}
/**
@@ -258,10 +258,10 @@ public static ChannelBuffer writeMessageTcp(XDR request, boolean last) {
* @param response XDR response
* @return UDP buffer
*/
- public static ChannelBuffer writeMessageUdp(XDR response) {
+ public static ByteBuf writeMessageUdp(XDR response) {
Preconditions.checkState(response.state == XDR.State.READING);
// TODO: Investigate whether making a copy of the buffer is necessary.
- return ChannelBuffers.copiedBuffer(response.buf);
+ return Unpooled.copiedBuffer(response.buf);
}
public static int fragmentSize(byte[] mark) {
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java
index 123999d5e14c0..a60928471e46b 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java
@@ -22,21 +22,27 @@
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.group.DefaultChannelGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioDatagramChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.logging.LogLevel;
+import io.netty.handler.logging.LoggingHandler;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.concurrent.GlobalEventExecutor;
import org.apache.hadoop.oncrpc.RpcProgram;
import org.apache.hadoop.oncrpc.RpcUtil;
import org.apache.hadoop.util.StringUtils;
-import org.jboss.netty.bootstrap.ConnectionlessBootstrap;
-import org.jboss.netty.bootstrap.ServerBootstrap;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.channel.group.DefaultChannelGroup;
-import org.jboss.netty.channel.socket.nio.NioDatagramChannelFactory;
-import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
-import org.jboss.netty.handler.timeout.IdleStateHandler;
-import org.jboss.netty.util.HashedWheelTimer;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
@@ -49,11 +55,17 @@ final class Portmap {
private static final Logger LOG = LoggerFactory.getLogger(Portmap.class);
private static final int DEFAULT_IDLE_TIME_MILLISECONDS = 5000;
- private ConnectionlessBootstrap udpServer;
+ private Bootstrap udpServer;
private ServerBootstrap tcpServer;
- private ChannelGroup allChannels = new DefaultChannelGroup();
+ private ChannelGroup allChannels = new DefaultChannelGroup(
+ GlobalEventExecutor.INSTANCE);
private Channel udpChannel;
private Channel tcpChannel;
+
+ EventLoopGroup bossGroup;
+ EventLoopGroup workerGroup;
+ EventLoopGroup udpGroup;
+
private final RpcProgramPortmap handler = new RpcProgramPortmap(allChannels);
public static void main(String[] args) {
@@ -73,18 +85,19 @@ public static void main(String[] args) {
void shutdown() {
allChannels.close().awaitUninterruptibly();
- tcpServer.releaseExternalResources();
- udpServer.releaseExternalResources();
+ bossGroup.shutdownGracefully();
+ workerGroup.shutdownGracefully();
+ udpGroup.shutdownGracefully();
}
@VisibleForTesting
SocketAddress getTcpServerLocalAddress() {
- return tcpChannel.getLocalAddress();
+ return tcpChannel.localAddress();
}
@VisibleForTesting
SocketAddress getUdpServerLoAddress() {
- return udpChannel.getLocalAddress();
+ return udpChannel.localAddress();
}
@VisibleForTesting
@@ -93,38 +106,55 @@ RpcProgramPortmap getHandler() {
}
void start(final int idleTimeMilliSeconds, final SocketAddress tcpAddress,
- final SocketAddress udpAddress) {
-
- tcpServer = new ServerBootstrap(new NioServerSocketChannelFactory(
- Executors.newCachedThreadPool(), Executors.newCachedThreadPool()));
- tcpServer.setPipelineFactory(new ChannelPipelineFactory() {
- private final HashedWheelTimer timer = new HashedWheelTimer();
- private final IdleStateHandler idleStateHandler = new IdleStateHandler(
- timer, 0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS);
-
- @Override
- public ChannelPipeline getPipeline() throws Exception {
- return Channels.pipeline(RpcUtil.constructRpcFrameDecoder(),
- RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler,
- RpcUtil.STAGE_RPC_TCP_RESPONSE);
- }
- });
- tcpServer.setOption("reuseAddress", true);
- tcpServer.setOption("child.reuseAddress", true);
-
- udpServer = new ConnectionlessBootstrap(new NioDatagramChannelFactory(
- Executors.newCachedThreadPool()));
-
- udpServer.setPipeline(Channels.pipeline(RpcUtil.STAGE_RPC_MESSAGE_PARSER,
- handler, RpcUtil.STAGE_RPC_UDP_RESPONSE));
- udpServer.setOption("reuseAddress", true);
-
- tcpChannel = tcpServer.bind(tcpAddress);
- udpChannel = udpServer.bind(udpAddress);
+ final SocketAddress udpAddress) throws InterruptedException {
+
+ bossGroup = new NioEventLoopGroup();
+ workerGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
+
+ tcpServer = new ServerBootstrap();
+ tcpServer.group(bossGroup, workerGroup)
+ .option(ChannelOption.SO_REUSEADDR, true)
+ .childOption(ChannelOption.SO_REUSEADDR, true)
+ .channel(NioServerSocketChannel.class)
+ .childHandler(new ChannelInitializer() {
+ private final IdleStateHandler idleStateHandler = new IdleStateHandler(
+ 0, 0, idleTimeMilliSeconds, TimeUnit.MILLISECONDS);
+
+ @Override
+ protected void initChannel(SocketChannel ch) throws Exception {
+ ChannelPipeline p = ch.pipeline();
+
+ p.addLast(RpcUtil.constructRpcFrameDecoder(),
+ RpcUtil.STAGE_RPC_MESSAGE_PARSER, idleStateHandler, handler,
+ RpcUtil.STAGE_RPC_TCP_RESPONSE);
+ }});
+
+ udpGroup = new NioEventLoopGroup(0, Executors.newCachedThreadPool());
+
+ udpServer = new Bootstrap();
+ udpServer.group(udpGroup)
+ .channel(NioDatagramChannel.class)
+ .handler(new ChannelInitializer() {
+ @Override protected void initChannel(NioDatagramChannel ch)
+ throws Exception {
+ ChannelPipeline p = ch.pipeline();
+ p.addLast(
+ new LoggingHandler(LogLevel.DEBUG),
+ RpcUtil.STAGE_RPC_MESSAGE_PARSER, handler, RpcUtil.STAGE_RPC_UDP_RESPONSE);
+ }
+ })
+ .option(ChannelOption.SO_REUSEADDR, true);
+
+ ChannelFuture tcpChannelFuture = null;
+ tcpChannelFuture = tcpServer.bind(tcpAddress);
+ ChannelFuture udpChannelFuture = udpServer.bind(udpAddress);
+ tcpChannel = tcpChannelFuture.sync().channel();
+ udpChannel = udpChannelFuture.sync().channel();
+
allChannels.add(tcpChannel);
allChannels.add(udpChannel);
- LOG.info("Portmap server started at tcp://" + tcpChannel.getLocalAddress()
- + ", udp://" + udpChannel.getLocalAddress());
+ LOG.info("Portmap server started at tcp://" + tcpChannel.localAddress()
+ + ", udp://" + udpChannel.localAddress());
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/RpcProgramPortmap.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/RpcProgramPortmap.java
index 0bc380f614c1c..7b33a644fbe76 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/RpcProgramPortmap.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/RpcProgramPortmap.java
@@ -19,6 +19,14 @@
import java.util.concurrent.ConcurrentHashMap;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.handler.timeout.IdleState;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.handler.timeout.IdleStateHandler;
import org.apache.hadoop.oncrpc.RpcAcceptedReply;
import org.apache.hadoop.oncrpc.RpcCall;
import org.apache.hadoop.oncrpc.RpcInfo;
@@ -27,20 +35,12 @@
import org.apache.hadoop.oncrpc.RpcUtil;
import org.apache.hadoop.oncrpc.XDR;
import org.apache.hadoop.oncrpc.security.VerifierNone;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelStateEvent;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.handler.timeout.IdleState;
-import org.jboss.netty.handler.timeout.IdleStateAwareChannelUpstreamHandler;
-import org.jboss.netty.handler.timeout.IdleStateEvent;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-final class RpcProgramPortmap extends IdleStateAwareChannelUpstreamHandler {
+@ChannelHandler.Sharable
+final class RpcProgramPortmap extends IdleStateHandler {
static final int PROGRAM = 100000;
static final int VERSION = 2;
@@ -60,6 +60,8 @@ final class RpcProgramPortmap extends IdleStateAwareChannelUpstreamHandler {
private final ChannelGroup allChannels;
RpcProgramPortmap(ChannelGroup allChannels) {
+ super(1, 1, 1);
+ // FIXME: set default idle timeout 1 second.
this.allChannels = allChannels;
PortmapMapping m = new PortmapMapping(PROGRAM, VERSION,
PortmapMapping.TRANSPORT_TCP, RpcProgram.RPCB_PORT);
@@ -151,14 +153,14 @@ private XDR dump(int xid, XDR in, XDR out) {
}
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+ public void channelRead(ChannelHandlerContext ctx, Object msg)
throws Exception {
- RpcInfo info = (RpcInfo) e.getMessage();
+ RpcInfo info = (RpcInfo) msg;
RpcCall rpcCall = (RpcCall) info.header();
final int portmapProc = rpcCall.getProcedure();
int xid = rpcCall.getXid();
- XDR in = new XDR(info.data().toByteBuffer().asReadOnlyBuffer(),
+ XDR in = new XDR(info.data().nioBuffer().asReadOnlyBuffer(),
XDR.State.READING);
XDR out = new XDR();
@@ -181,29 +183,29 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
reply.write(out);
}
- ChannelBuffer buf = ChannelBuffers.wrappedBuffer(out.asReadOnlyWrap()
+ ByteBuf buf = Unpooled.wrappedBuffer(out.asReadOnlyWrap()
.buffer());
RpcResponse rsp = new RpcResponse(buf, info.remoteAddress());
RpcUtil.sendRpcResponse(ctx, rsp);
}
@Override
- public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent e)
+ public void channelActive(ChannelHandlerContext ctx)
throws Exception {
- allChannels.add(e.getChannel());
+ allChannels.add(ctx.channel());
}
@Override
public void channelIdle(ChannelHandlerContext ctx, IdleStateEvent e)
throws Exception {
- if (e.getState() == IdleState.ALL_IDLE) {
- e.getChannel().close();
+ if (e.state() == IdleState.ALL_IDLE) {
+ ctx.channel().close();
}
}
@Override
- public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) {
- LOG.warn("Encountered ", e.getCause());
- e.getChannel().close();
+ public void exceptionCaught(ChannelHandlerContext ctx, Throwable t) {
+ LOG.warn("Encountered ", t);
+ ctx.channel().close();
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java b/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java
index 0e416b3738d20..6d103fdd781c6 100644
--- a/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java
+++ b/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java
@@ -22,19 +22,19 @@
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
-import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Random;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelException;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.oncrpc.RpcUtil.RpcFrameDecoder;
import org.apache.hadoop.oncrpc.security.CredentialsNone;
import org.apache.hadoop.oncrpc.security.VerifierNone;
import org.apache.hadoop.test.GenericTestUtils;
-import org.jboss.netty.buffer.ByteBufferBackedChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.channel.ChannelHandlerContext;
import org.junit.Test;
import org.mockito.Mockito;
import org.slf4j.event.Level;
@@ -55,6 +55,7 @@ static void testRequest(XDR request, int serverPort) {
tcpClient.run();
}
+ @ChannelHandler.Sharable
static class TestRpcProgram extends RpcProgram {
protected TestRpcProgram(String program, String host, int port,
@@ -83,7 +84,7 @@ protected void handleInternal(ChannelHandlerContext ctx, RpcInfo info) {
new VerifierNone());
XDR out = new XDR();
reply.write(out);
- ChannelBuffer b = ChannelBuffers.wrappedBuffer(out.asReadOnlyWrap().buffer());
+ ByteBuf b = Unpooled.wrappedBuffer(out.asReadOnlyWrap().buffer());
RpcResponse rsp = new RpcResponse(b, info.remoteAddress());
RpcUtil.sendRpcResponse(ctx, rsp);
}
@@ -99,13 +100,14 @@ public void testSingleFrame() {
RpcFrameDecoder decoder = new RpcFrameDecoder();
// Test "Length field is not received yet"
- ByteBuffer buffer = ByteBuffer.allocate(1);
- ChannelBuffer buf = new ByteBufferBackedChannelBuffer(buffer);
- ChannelBuffer channelBuffer = (ChannelBuffer) decoder.decode(
- Mockito.mock(ChannelHandlerContext.class), Mockito.mock(Channel.class),
- buf);
- assertTrue(channelBuffer == null);
+ ByteBuf buf = Unpooled.directBuffer(1);
+ List outputBufs = new ArrayList<>();
+ decoder.decode(
+ Mockito.mock(ChannelHandlerContext.class), buf,
+ outputBufs);
+ assertTrue(outputBufs.isEmpty());
+ decoder = new RpcFrameDecoder();
// Test all bytes are not received yet
byte[] fragment = new byte[4 + 9];
fragment[0] = (byte) (1 << 7); // final fragment
@@ -114,15 +116,16 @@ public void testSingleFrame() {
fragment[3] = (byte) 10; // fragment size = 10 bytes
assertTrue(XDR.isLastFragment(fragment));
assertTrue(XDR.fragmentSize(fragment)==10);
+ buf.release();
- buffer = ByteBuffer.allocate(4 + 9);
- buffer.put(fragment);
- buffer.flip();
- buf = new ByteBufferBackedChannelBuffer(buffer);
- channelBuffer = (ChannelBuffer) decoder.decode(
- Mockito.mock(ChannelHandlerContext.class), Mockito.mock(Channel.class),
- buf);
- assertTrue(channelBuffer == null);
+ buf = Unpooled.directBuffer(4 + 9);
+ buf.writeBytes(fragment);
+ outputBufs = new ArrayList<>();
+ decoder.decode(
+ Mockito.mock(ChannelHandlerContext.class), buf,
+ outputBufs);
+ assertTrue(decoder.isLast());
+ buf.release();
}
@Test
@@ -137,16 +140,15 @@ public void testMultipleFrames() {
fragment1[3] = (byte) 10; // fragment size = 10 bytes
assertFalse(XDR.isLastFragment(fragment1));
assertTrue(XDR.fragmentSize(fragment1)==10);
+
+ List outputBufs = new ArrayList<>();
// decoder should wait for the final fragment
- ByteBuffer buffer = ByteBuffer.allocate(4 + 10);
- buffer.put(fragment1);
- buffer.flip();
- ChannelBuffer buf = new ByteBufferBackedChannelBuffer(buffer);
- ChannelBuffer channelBuffer = (ChannelBuffer) decoder.decode(
- Mockito.mock(ChannelHandlerContext.class), Mockito.mock(Channel.class),
- buf);
- assertTrue(channelBuffer == null);
+ ByteBuf buf = Unpooled.directBuffer(4 + 10, 4 + 10);
+ buf.writeBytes(fragment1);
+ decoder.decode(
+ Mockito.mock(ChannelHandlerContext.class), buf,
+ outputBufs);
byte[] fragment2 = new byte[4 + 10];
fragment2[0] = (byte) (1 << 7); // final fragment
@@ -155,21 +157,22 @@ public void testMultipleFrames() {
fragment2[3] = (byte) 10; // fragment size = 10 bytes
assertTrue(XDR.isLastFragment(fragment2));
assertTrue(XDR.fragmentSize(fragment2)==10);
+ buf.release();
- buffer = ByteBuffer.allocate(4 + 10);
- buffer.put(fragment2);
- buffer.flip();
- buf = new ByteBufferBackedChannelBuffer(buffer);
- channelBuffer = (ChannelBuffer) decoder.decode(
- Mockito.mock(ChannelHandlerContext.class), Mockito.mock(Channel.class),
- buf);
- assertTrue(channelBuffer != null);
- // Complete frame should have to total size 10+10=20
- assertEquals(20, channelBuffer.readableBytes());
+ buf = Unpooled.directBuffer(4 + 10, 4 + 10);
+ buf.writeBytes(fragment2);
+ decoder.decode(
+ Mockito.mock(ChannelHandlerContext.class), buf,
+ outputBufs);
+ // Expect two completed frames each 10 bytes
+ decoder.isLast();
+ assertEquals(2, outputBufs.size());
+ outputBufs.forEach(b -> assertEquals(((ByteBuf)b).readableBytes(), 10));
+ buf.release();
}
@Test
- public void testFrames() {
+ public void testFrames() throws InterruptedException {
int serverPort = startRpcServer(true);
XDR xdrOut = createGetportMount();
@@ -187,7 +190,7 @@ public void testFrames() {
}
@Test
- public void testUnprivilegedPort() {
+ public void testUnprivilegedPort() throws InterruptedException {
// Don't allow connections from unprivileged ports. Given that this test is
// presumably not being run by root, this will be the case.
int serverPort = startRpcServer(false);
@@ -218,23 +221,28 @@ public void testUnprivilegedPort() {
assertEquals(requestSize, resultSize);
}
- private static int startRpcServer(boolean allowInsecurePorts) {
+ private static int startRpcServer(boolean allowInsecurePorts)
+ throws InterruptedException {
Random rand = new Random();
int serverPort = 30000 + rand.nextInt(10000);
int retries = 10; // A few retries in case initial choice is in use.
while (true) {
+ SimpleTcpServer tcpServer = null;
try {
RpcProgram program = new TestFrameDecoder.TestRpcProgram("TestRpcProgram",
"localhost", serverPort, 100000, 1, 2, allowInsecurePorts);
- SimpleTcpServer tcpServer = new SimpleTcpServer(serverPort, program, 1);
+ tcpServer = new SimpleTcpServer(serverPort, program, 1);
tcpServer.run();
break; // Successfully bound a port, break out.
- } catch (ChannelException ce) {
+ } catch (InterruptedException | ChannelException e) {
+ if (tcpServer != null) {
+ tcpServer.shutdown();
+ }
if (retries-- > 0) {
serverPort += rand.nextInt(20); // Port in use? Try another.
} else {
- throw ce; // Out of retries.
+ throw e; // Out of retries.
}
}
}
diff --git a/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java b/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java
index 6941c4a04e998..8ebf9d03c6c30 100644
--- a/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java
+++ b/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java
@@ -43,7 +43,7 @@ public class TestPortmap {
private int xid;
@BeforeClass
- public static void setup() {
+ public static void setup() throws InterruptedException {
pm.start(SHORT_TIMEOUT_MILLISECONDS, new InetSocketAddress("localhost", 0),
new InetSocketAddress("localhost", 0));
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
index 6ef1586647404..667ad083eb17c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@ -89,10 +89,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
io.nettynetty-common
+
+ io.netty
+ netty-codec-socks
+ io.nettynetty-handler
+
+ io.netty
+ netty-handler-proxy
+ io.nettynetty-transport
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
index 2bed0913f8ce7..1553adf0b6a64 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
@@ -377,7 +377,7 @@
-
+
dist
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
index 6328943981a55..455b54cba3b79 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
@@ -47,7 +47,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
io.netty
- netty
+ netty-allcompile
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
index 27213953802f7..2dc50db54833e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
@@ -26,6 +26,10 @@
import java.util.List;
import java.util.HashMap;
+import io.netty.channel.ChannelHandler;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelHandlerContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FileSystem;
@@ -51,15 +55,13 @@
import org.apache.hadoop.oncrpc.security.VerifierNone;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.ChannelHandlerContext;
import com.google.common.annotations.VisibleForTesting;
/**
* RPC program corresponding to mountd daemon. See {@link Mountd}.
*/
+@ChannelHandler.Sharable
public class RpcProgramMountd extends RpcProgram implements MountInterface {
private static final Logger LOG =
LoggerFactory.getLogger(RpcProgramMountd.class);
@@ -262,8 +264,8 @@ RpcAcceptedReply.AcceptState.PROC_UNAVAIL, new VerifierNone())
RpcAcceptedReply.AcceptState.PROC_UNAVAIL, new VerifierNone()).write(
out);
}
- ChannelBuffer buf =
- ChannelBuffers.wrappedBuffer(out.asReadOnlyWrap().buffer());
+ ByteBuf buf =
+ Unpooled.wrappedBuffer(out.asReadOnlyWrap().buffer());
RpcResponse rsp = new RpcResponse(buf, info.remoteAddress());
RpcUtil.sendRpcResponse(ctx, rsp);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
index c6da1981f3716..c58dc5976b37d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
@@ -22,6 +22,8 @@
import java.net.URI;
import java.nio.file.FileSystemException;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsConstants;
@@ -39,8 +41,6 @@
import org.apache.hadoop.nfs.nfs3.response.WccData;
import org.apache.hadoop.oncrpc.XDR;
import org.apache.hadoop.security.IdMappingServiceProvider;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.Channel;
/**
* Utility/helper methods related to NFS
@@ -147,16 +147,16 @@ public static void writeChannel(Channel channel, XDR out, int xid) {
if (RpcProgramNfs3.LOG.isDebugEnabled()) {
RpcProgramNfs3.LOG.debug(WRITE_RPC_END + xid);
}
- ChannelBuffer outBuf = XDR.writeMessageTcp(out, true);
- channel.write(outBuf);
+ ByteBuf outBuf = XDR.writeMessageTcp(out, true);
+ channel.writeAndFlush(outBuf);
}
public static void writeChannelCommit(Channel channel, XDR out, int xid) {
if (RpcProgramNfs3.LOG.isDebugEnabled()) {
RpcProgramNfs3.LOG.debug("Commit done:" + xid);
}
- ChannelBuffer outBuf = XDR.writeMessageTcp(out, true);
- channel.write(outBuf);
+ ByteBuf outBuf = XDR.writeMessageTcp(out, true);
+ channel.writeAndFlush(outBuf);
}
private static boolean isSet(int access, int bits) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
index 6067a5df34786..ca29db984dc42 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
@@ -31,6 +31,7 @@
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicLong;
+import io.netty.channel.Channel;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
@@ -55,7 +56,6 @@
import org.apache.hadoop.security.IdMappingServiceProvider;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
-import org.jboss.netty.channel.Channel;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
index ea5cdceffd947..6d95976efcc63 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
@@ -28,6 +28,11 @@
import java.nio.charset.Charset;
import java.util.EnumSet;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
@@ -129,10 +134,6 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AuthorizationException;
import org.apache.hadoop.util.JvmPauseMonitor;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
@@ -141,6 +142,7 @@
/**
* RPC program corresponding to nfs daemon. See {@link Nfs3}.
*/
+@ChannelHandler.Sharable
public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
public static final int DEFAULT_UMASK = 0022;
public static final FsPermission umask = new FsPermission(
@@ -2180,7 +2182,7 @@ public void handleInternal(ChannelHandlerContext ctx, RpcInfo info) {
RpcDeniedReply.RejectState.AUTH_ERROR, new VerifierNone());
rdr.write(reply);
- ChannelBuffer buf = ChannelBuffers.wrappedBuffer(reply.asReadOnlyWrap()
+ ByteBuf buf = Unpooled.wrappedBuffer(reply.asReadOnlyWrap()
.buffer());
RpcResponse rsp = new RpcResponse(buf, info.remoteAddress());
RpcUtil.sendRpcResponse(ctx, rsp);
@@ -2291,7 +2293,7 @@ RpcAcceptedReply.AcceptState.PROC_UNAVAIL, new VerifierNone()).write(
}
// TODO: currently we just return VerifierNone
out = response.serialize(out, xid, new VerifierNone());
- ChannelBuffer buf = ChannelBuffers.wrappedBuffer(out.asReadOnlyWrap()
+ ByteBuf buf = Unpooled.wrappedBuffer(out.asReadOnlyWrap()
.buffer());
RpcResponse rsp = new RpcResponse(buf, info.remoteAddress());
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
index 98f3d6cfa2930..37d4bbe1feb68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java
@@ -22,12 +22,12 @@
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
+import io.netty.channel.Channel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.nfs.nfs3.FileHandle;
import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
-import org.jboss.netty.channel.Channel;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java
index 35542391bdb4c..ec50893968592 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.util.EnumSet;
+import io.netty.channel.Channel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -43,7 +44,6 @@
import org.apache.hadoop.oncrpc.XDR;
import org.apache.hadoop.oncrpc.security.VerifierNone;
import org.apache.hadoop.security.IdMappingServiceProvider;
-import org.jboss.netty.channel.Channel;
import com.google.common.annotations.VisibleForTesting;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
index 4e53c72bec8a8..31528a2db87a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestOutOfOrderWrite.java
@@ -21,6 +21,12 @@
import java.nio.ByteBuffer;
import java.util.Arrays;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.socket.SocketChannel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
@@ -42,13 +48,6 @@
import org.apache.hadoop.oncrpc.XDR;
import org.apache.hadoop.oncrpc.security.CredentialsNone;
import org.apache.hadoop.oncrpc.security.VerifierNone;
-import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.MessageEvent;
public class TestOutOfOrderWrite {
public final static Logger LOG =
@@ -100,9 +99,9 @@ public WriteHandler(XDR request) {
}
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
+ public void channelRead(ChannelHandlerContext ctx, Object msg) {
// Get handle from create response
- ChannelBuffer buf = (ChannelBuffer) e.getMessage();
+ ByteBuf buf = (ByteBuf) msg;
XDR rsp = new XDR(buf.array());
if (rsp.getBytes().length == 0) {
LOG.info("rsp length is zero, why?");
@@ -125,7 +124,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
rsp.readBoolean(); // value follow
handle = new FileHandle();
handle.deserialize(rsp);
- channel = e.getChannel();
+ channel = ctx.channel();
}
}
@@ -136,16 +135,17 @@ public WriteClient(String host, int port, XDR request, Boolean oneShot) {
}
@Override
- protected ChannelPipelineFactory setPipelineFactory() {
- this.pipelineFactory = new ChannelPipelineFactory() {
+ protected ChannelInitializer setChannelHandler() {
+ return new ChannelInitializer() {
@Override
- public ChannelPipeline getPipeline() {
- return Channels.pipeline(
+ protected void initChannel(SocketChannel ch) throws Exception {
+ ChannelPipeline p = ch.pipeline();
+ p.addLast(
RpcUtil.constructRpcFrameDecoder(),
- new WriteHandler(request));
+ new WriteHandler(request)
+ );
}
};
- return this.pipelineFactory;
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
index 30ecc0b824b9e..07954c00d64e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
@@ -28,6 +28,7 @@
import java.nio.ByteBuffer;
import java.util.EnumSet;
+import io.netty.channel.Channel;
import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -92,7 +93,6 @@
import org.apache.hadoop.security.IdMappingConstant;
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.jboss.netty.channel.Channel;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
index f7a92fac53501..0f03c6da93bf3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
@@ -27,6 +27,7 @@
import java.util.Arrays;
import java.util.concurrent.ConcurrentNavigableMap;
+import io.netty.channel.Channel;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -52,7 +53,6 @@
import org.apache.hadoop.security.ShellBasedIdMapping;
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.jboss.netty.channel.Channel;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterClientRejectOverload.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterClientRejectOverload.java
index 3c51e13182c92..3aed20754fe83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterClientRejectOverload.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterClientRejectOverload.java
@@ -48,7 +48,7 @@
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
+import com.fasterxml.jackson.databind.ObjectMapper;
/**
* Test the Router overload control which rejects requests when the RPC client
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index d2e8690a6cb76..6728194714775 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -160,7 +160,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
io.netty
- netty
+ netty-allcompile
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
index 341a2f01173d3..f847f5f0c4b52 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
@@ -58,10 +58,10 @@
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Timer;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.ObjectReader;
-import org.codehaus.jackson.map.ObjectWriter;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
import com.google.common.annotations.VisibleForTesting;
@@ -373,14 +373,11 @@ public void releaseReservedSpace(long bytesToRelease) {
private static final ObjectWriter WRITER =
new ObjectMapper().writerWithDefaultPrettyPrinter();
- private static final ObjectReader READER =
- new ObjectMapper().reader(ProvidedBlockIteratorState.class);
private static class ProvidedBlockIteratorState {
ProvidedBlockIteratorState() {
iterStartMs = Time.now();
lastSavedMs = iterStartMs;
- atEnd = false;
lastBlockId = -1;
}
@@ -392,9 +389,6 @@ private static class ProvidedBlockIteratorState {
@JsonProperty
private long iterStartMs;
- @JsonProperty
- private boolean atEnd;
-
// The id of the last block read when the state of the iterator is saved.
// This implementation assumes that provided blocks are returned
// in sorted order of the block ids.
diff --git a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
index 4e459b652b29c..a9f69eb91d66b 100644
--- a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
+++ b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
@@ -291,6 +291,14 @@
+
+
+
+
+
+
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index bb650fe6be83e..7f28cf2cd2312 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -112,6 +112,19 @@
maven-antrun-plugin
+ generate-sources-ant-replace
+ generate-sources
+
+ run
+
+
+
+
+
+
+
+
+ pre-site-ant-copypre-siterun
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
index a1447c58b4631..3818c1bd3d5f2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.util.StringUtils;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AvroArrayUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AvroArrayUtils.java
index 608756657d55c..d9e7a70750ab1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AvroArrayUtils.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AvroArrayUtils.java
@@ -22,8 +22,8 @@
import java.util.Iterator;
import java.util.List;
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.thirdparty.avro.Schema;
+import org.apache.hadoop.thirdparty.avro.generic.GenericData;
public class AvroArrayUtils {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java
index 3c7055ce33a60..0c4e00258bd22 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java
@@ -23,13 +23,13 @@
import java.io.EOFException;
import java.io.IOException;
-import org.apache.avro.AvroRuntimeException;
-import org.apache.avro.Schema;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.Decoder;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.specific.SpecificData;
-import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.hadoop.thirdparty.avro.AvroRuntimeException;
+import org.apache.hadoop.thirdparty.avro.Schema;
+import org.apache.hadoop.thirdparty.avro.io.DatumReader;
+import org.apache.hadoop.thirdparty.avro.io.Decoder;
+import org.apache.hadoop.thirdparty.avro.io.DecoderFactory;
+import org.apache.hadoop.thirdparty.avro.specific.SpecificData;
+import org.apache.hadoop.thirdparty.avro.specific.SpecificDatumReader;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventWriter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventWriter.java
index 02ec53e559c0f..75afdc112be2d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventWriter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventWriter.java
@@ -21,12 +21,12 @@
import java.io.IOException;
import java.util.ArrayList;
-import org.apache.avro.Schema;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.Encoder;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.avro.specific.SpecificDatumWriter;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.Schema;
+import org.apache.hadoop.thirdparty.avro.io.DatumWriter;
+import org.apache.hadoop.thirdparty.avro.io.Encoder;
+import org.apache.hadoop.thirdparty.avro.io.EncoderFactory;
+import org.apache.hadoop.thirdparty.avro.specific.SpecificDatumWriter;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FSDataOutputStream;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
index 34e4b2c89c8f4..493bdb909dafc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.Counters;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
index f5941aa04479d..639770a318909 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.JobID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
index 784267f2b0a70..efa61403145e1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.JobID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
index 1616dd1962b2b..42b2bf6b0f054 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.JobPriority;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
index 66f378123986d..4ac9f7a294ea4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
@@ -20,8 +20,8 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
@@ -31,8 +31,8 @@ public class JobQueueChangeEvent implements HistoryEvent {
private JobQueueChange datum = new JobQueueChange();
public JobQueueChangeEvent(JobID id, String queueName) {
- datum.jobid = new Utf8(id.toString());
- datum.jobQueueName = new Utf8(queueName);
+ datum.setJobid(new Utf8(id.toString()));
+ datum.setJobQueueName(new Utf8(queueName));
}
JobQueueChangeEvent() { }
@@ -54,13 +54,13 @@ public void setDatum(Object datum) {
/** Get the Job ID */
public JobID getJobId() {
- return JobID.forName(datum.jobid.toString());
+ return JobID.forName(datum.getJobid().toString());
}
/** Get the new Job queue name */
public String getJobQueueName() {
- if (datum.jobQueueName != null) {
- return datum.jobQueueName.toString();
+ if (datum.getJobQueueName() != null) {
+ return datum.getJobQueueName().toString();
}
return null;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
index 0963b45b492bd..302072b1a1c20 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.JobID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
index e394f5bbf6ec2..4564beaa1d79c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
@@ -23,7 +23,7 @@
import java.util.Map.Entry;
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.JobConf;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
index da31591a26d42..b1e0420e54ba7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
@@ -21,7 +21,7 @@
import java.util.Collections;
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.JobID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
index 2b1357ea85902..d80d0cd28590b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.ProgressSplitsBlock;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
index 5a16f834acb4a..916b01b7bc3a2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.ProgressSplitsBlock;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
index c28c21605df86..2c9c46f0a3e36 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.Counters;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
index d09d5ca6e9b8a..48690f854a6db 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.TaskAttemptID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
index 9afa09384cc9e..4154cc73eae49 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.ProgressSplitsBlock;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
index b4d9e410da27d..ec796a36d1cab 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.TaskStatus;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
index 97557c7e0b4fe..eb4588926292d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.TaskStatus;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
index 9d2fc0eee5541..ff829d6ec4d95 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.TaskID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
index 010129d73ceb4..258e7929c13b4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
@@ -20,7 +20,7 @@
import java.util.Set;
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.thirdparty.avro.util.Utf8;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.TaskID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
index 48396e1b53694..1f25b534dc644 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
@@ -53,14 +53,15 @@
import com.google.common.annotations.VisibleForTesting;
-class Fetcher extends Thread {
+@VisibleForTesting
+public class Fetcher extends Thread {
private static final Logger LOG = LoggerFactory.getLogger(Fetcher.class);
- /** Number of ms before timing out a copy */
+ /** Number of ms before timing out a copy. */
private static final int DEFAULT_STALLED_COPY_TIMEOUT = 3 * 60 * 1000;
- /** Basic/unit connection timeout (in milliseconds) */
+ /** Basic/unit connection timeout (in milliseconds). */
private final static int UNIT_CONNECT_TIMEOUT = 60 * 1000;
/* Default read timeout (in milliseconds) */
@@ -72,10 +73,12 @@ class Fetcher extends Thread {
private static final String FETCH_RETRY_AFTER_HEADER = "Retry-After";
protected final Reporter reporter;
- private enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
+ @VisibleForTesting
+ public enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
CONNECTION, WRONG_REDUCE}
-
- private final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors";
+
+ @VisibleForTesting
+ public final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors";
private final JobConf jobConf;
private final Counters.Counter connectionErrs;
private final Counters.Counter ioErrs;
@@ -83,8 +86,8 @@ private enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
private final Counters.Counter badIdErrs;
private final Counters.Counter wrongMapErrs;
private final Counters.Counter wrongReduceErrs;
- protected final MergeManager merger;
- protected final ShuffleSchedulerImpl scheduler;
+ protected final MergeManager merger;
+ protected final ShuffleSchedulerImpl scheduler;
protected final ShuffleClientMetrics metrics;
protected final ExceptionReporter exceptionReporter;
protected final int id;
@@ -111,7 +114,7 @@ private enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
private static SSLFactory sslFactory;
public Fetcher(JobConf job, TaskAttemptID reduceId,
- ShuffleSchedulerImpl scheduler, MergeManager merger,
+ ShuffleSchedulerImpl scheduler, MergeManager merger,
Reporter reporter, ShuffleClientMetrics metrics,
ExceptionReporter exceptionReporter, SecretKey shuffleKey) {
this(job, reduceId, scheduler, merger, reporter, metrics,
@@ -120,7 +123,7 @@ public Fetcher(JobConf job, TaskAttemptID reduceId,
@VisibleForTesting
Fetcher(JobConf job, TaskAttemptID reduceId,
- ShuffleSchedulerImpl scheduler, MergeManager merger,
+ ShuffleSchedulerImpl scheduler, MergeManager merger,
Reporter reporter, ShuffleClientMetrics metrics,
ExceptionReporter exceptionReporter, SecretKey shuffleKey,
int id) {
@@ -315,9 +318,8 @@ protected void copyFromHost(MapHost host) throws IOException {
return;
}
- if(LOG.isDebugEnabled()) {
- LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: "
- + maps);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: " + maps);
}
// List of maps to be fetched yet
@@ -411,8 +413,8 @@ private void openConnectionWithRetry(URL url) throws IOException {
shouldWait = false;
} catch (IOException e) {
if (!fetchRetryEnabled) {
- // throw exception directly if fetch's retry is not enabled
- throw e;
+ // throw exception directly if fetch's retry is not enabled
+ throw e;
}
if ((Time.monotonicNow() - startTime) >= this.fetchRetryTimeout) {
LOG.warn("Failed to connect to host: " + url + "after "
@@ -489,7 +491,7 @@ private TaskAttemptID[] copyMapOutput(MapHost host,
DataInputStream input,
Set remaining,
boolean canRetry) throws IOException {
- MapOutput mapOutput = null;
+ MapOutput mapOutput = null;
TaskAttemptID mapId = null;
long decompressedLength = -1;
long compressedLength = -1;
@@ -611,7 +613,7 @@ private void checkTimeoutOrRetry(MapHost host, IOException ioe)
// First time to retry.
long currentTime = Time.monotonicNow();
if (retryStartTime == 0) {
- retryStartTime = currentTime;
+ retryStartTime = currentTime;
}
// Retry is not timeout, let's do retry with throwing an exception.
@@ -628,7 +630,7 @@ private void checkTimeoutOrRetry(MapHost host, IOException ioe)
}
/**
- * Do some basic verification on the input received -- Being defensive
+ * Do some basic verification on the input received -- Being defensive.
* @param compressedLength
* @param decompressedLength
* @param forReduce
@@ -695,8 +697,7 @@ private URL getMapOutputURL(MapHost host, Collection maps
* only on the last failure. Instead of connecting with a timeout of
* X, we try connecting with a timeout of x < X but multiple times.
*/
- private void connect(URLConnection connection, int connectionTimeout)
- throws IOException {
+ private void connect(URLConnection connection, int connectionTimeout) throws IOException {
int unit = 0;
if (connectionTimeout < 0) {
throw new IOException("Invalid timeout "
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java
index 9b04f64ac6042..1b99ce0c0aa1d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java
@@ -26,6 +26,7 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.task.reduce.Fetcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -37,6 +38,7 @@
import java.util.Formatter;
import java.util.Iterator;
+import static org.apache.hadoop.mapreduce.task.reduce.Fetcher.SHUFFLE_ERR_GRP_NAME;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -87,6 +89,9 @@ public void testReduceFromPartialMem() throws Exception {
final long spill = c.findCounter(TaskCounter.SPILLED_RECORDS).getCounter();
assertTrue("Expected some records not spilled during reduce" + spill + ")",
spill < 2 * out); // spilled map records, some records at the reduce
+ long shuffleIoErrors =
+ c.getGroup(SHUFFLE_ERR_GRP_NAME).getCounter(Fetcher.ShuffleErrors.IO_ERROR.toString());
+ assertEquals(0, shuffleIoErrors);
}
/**
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
index 64cb253d9c587..1bd563e822027 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
@@ -25,7 +25,7 @@
import org.junit.Assert;
-import org.apache.avro.AvroRemoteException;
+import org.apache.hadoop.thirdparty.avro.AvroRemoteException;
import org.apache.hadoop.mapreduce.SleepJob;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
index d97da9b2111ac..05f88105c8532 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/pom.xml
@@ -41,8 +41,18 @@
hadoop-mapreduce-client-core
- org.apache.avro
- avro
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-avro_1_11
+
+
+ org.slf4j
+ slf4j-api
+
+
+ org.xerial.snappy
+ snappy-java
+
+ test
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java
index e9f0f34c69e95..1f009a4919576 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedChunkedFile.java
@@ -22,7 +22,10 @@
import java.io.IOException;
import java.io.RandomAccessFile;
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.VisibleForTesting;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.stream.ChunkedFile;
import org.apache.hadoop.io.ReadaheadPool;
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
import org.apache.hadoop.io.nativeio.NativeIO;
@@ -31,8 +34,6 @@
import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
-import org.jboss.netty.handler.stream.ChunkedFile;
-
public class FadvisedChunkedFile extends ChunkedFile {
private static final Logger LOG =
@@ -64,16 +65,16 @@ FileDescriptor getFd() {
}
@Override
- public Object nextChunk() throws Exception {
+ public ByteBuf readChunk(ByteBufAllocator allocator) throws Exception {
synchronized (closeLock) {
if (fd.valid()) {
if (manageOsCache && readaheadPool != null) {
readaheadRequest = readaheadPool
.readaheadStream(
- identifier, fd, getCurrentOffset(), readaheadLength,
- getEndOffset(), readaheadRequest);
+ identifier, fd, currentOffset(), readaheadLength,
+ endOffset(), readaheadRequest);
}
- return super.nextChunk();
+ return super.readChunk(allocator);
} else {
return null;
}
@@ -88,12 +89,12 @@ public void close() throws Exception {
readaheadRequest = null;
}
if (fd.valid() &&
- manageOsCache && getEndOffset() - getStartOffset() > 0) {
+ manageOsCache && endOffset() - startOffset() > 0) {
try {
NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
identifier,
fd,
- getStartOffset(), getEndOffset() - getStartOffset(),
+ startOffset(), endOffset() - startOffset(),
POSIX_FADV_DONTNEED);
} catch (Throwable t) {
LOG.warn("Failed to manage OS cache for " + identifier +
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java
index 4b2c8cbf20afd..9290a282e3917 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java
@@ -25,6 +25,7 @@
import java.nio.channels.FileChannel;
import java.nio.channels.WritableByteChannel;
+import io.netty.channel.DefaultFileRegion;
import org.apache.hadoop.io.ReadaheadPool;
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
import org.apache.hadoop.io.nativeio.NativeIO;
@@ -33,9 +34,7 @@
import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
-import org.jboss.netty.channel.DefaultFileRegion;
-
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.VisibleForTesting;
public class FadvisedFileRegion extends DefaultFileRegion {
@@ -77,8 +76,8 @@ public long transferTo(WritableByteChannel target, long position)
throws IOException {
if (readaheadPool != null && readaheadLength > 0) {
readaheadRequest = readaheadPool.readaheadStream(identifier, fd,
- getPosition() + position, readaheadLength,
- getPosition() + getCount(), readaheadRequest);
+ position() + position, readaheadLength,
+ position() + count(), readaheadRequest);
}
if(this.shuffleTransferToAllowed) {
@@ -147,11 +146,11 @@ long customShuffleTransfer(WritableByteChannel target, long position)
@Override
- public void releaseExternalResources() {
+ protected void deallocate() {
if (readaheadRequest != null) {
readaheadRequest.cancel();
}
- super.releaseExternalResources();
+ super.deallocate();
}
/**
@@ -159,10 +158,10 @@ public void releaseExternalResources() {
* we don't need the region to be cached anymore.
*/
public void transferSuccessful() {
- if (manageOsCache && getCount() > 0) {
+ if (manageOsCache && count() > 0) {
try {
NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
- fd, getPosition(), getCount(), POSIX_FADV_DONTNEED);
+ fd, position(), count(), POSIX_FADV_DONTNEED);
} catch (Throwable t) {
LOG.warn("Failed to manage OS cache for " + identifier, t);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/LoggingHttpResponseEncoder.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/LoggingHttpResponseEncoder.java
new file mode 100644
index 0000000000000..c7b98ce166ca6
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/LoggingHttpResponseEncoder.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPromise;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseEncoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+class LoggingHttpResponseEncoder extends HttpResponseEncoder {
+ private static final Logger LOG = LoggerFactory.getLogger(LoggingHttpResponseEncoder.class);
+ private final boolean logStacktraceOfEncodingMethods;
+
+ LoggingHttpResponseEncoder(boolean logStacktraceOfEncodingMethods) {
+ this.logStacktraceOfEncodingMethods = logStacktraceOfEncodingMethods;
+ }
+
+ @Override
+ public boolean acceptOutboundMessage(Object msg) throws Exception {
+ printExecutingMethod();
+ LOG.info("OUTBOUND MESSAGE: " + msg);
+ return super.acceptOutboundMessage(msg);
+ }
+
+ @Override
+ protected void encodeInitialLine(ByteBuf buf, HttpResponse response) throws Exception {
+ LOG.debug("Executing method: {}, response: {}",
+ getExecutingMethodName(), response);
+ logStacktraceIfRequired();
+ super.encodeInitialLine(buf, response);
+ }
+
+ @Override
+ protected void encode(ChannelHandlerContext ctx, Object msg,
+ List out) throws Exception {
+ LOG.debug("Encoding to channel {}: {}", ctx.channel(), msg);
+ printExecutingMethod();
+ logStacktraceIfRequired();
+ super.encode(ctx, msg, out);
+ }
+
+ @Override
+ protected void encodeHeaders(HttpHeaders headers, ByteBuf buf) {
+ printExecutingMethod();
+ super.encodeHeaders(headers, buf);
+ }
+
+ @Override
+ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise
+ promise) throws Exception {
+ LOG.debug("Writing to channel {}: {}", ctx.channel(), msg);
+ printExecutingMethod();
+ super.write(ctx, msg, promise);
+ }
+
+ private void logStacktraceIfRequired() {
+ if (logStacktraceOfEncodingMethods) {
+ LOG.debug("Stacktrace: ", new Throwable());
+ }
+ }
+
+ private void printExecutingMethod() {
+ String methodName = getExecutingMethodName(1);
+ LOG.debug("Executing method: {}", methodName);
+ }
+
+ private String getExecutingMethodName() {
+ return getExecutingMethodName(0);
+ }
+
+ private String getExecutingMethodName(int additionalSkipFrames) {
+ try {
+ StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace();
+ // Array items (indices):
+ // 0: java.lang.Thread.getStackTrace(...)
+ // 1: TestShuffleHandler$LoggingHttpResponseEncoder.getExecutingMethodName(...)
+ int skipFrames = 2 + additionalSkipFrames;
+ String methodName = stackTrace[skipFrames].getMethodName();
+ String className = this.getClass().getSimpleName();
+ return className + "#" + methodName;
+ } catch (Throwable t) {
+ LOG.error("Error while getting execution method name", t);
+ return "unknown";
+ }
+ }
+}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index eec8f040c4f86..fc755a67631f0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -18,19 +18,20 @@
package org.apache.hadoop.mapred;
+import static io.netty.buffer.Unpooled.wrappedBuffer;
+import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_TYPE;
+import static io.netty.handler.codec.http.HttpMethod.GET;
+import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
+import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;
+import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+import static org.apache.hadoop.mapred.ShuffleHandler.NettyChannelHelper.*;
import static org.fusesource.leveldbjni.JniDBFactory.asString;
import static org.fusesource.leveldbjni.JniDBFactory.bytes;
-import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;
-import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import java.io.File;
import java.io.FileNotFoundException;
@@ -54,6 +55,44 @@
import javax.crypto.SecretKey;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelOutboundHandlerAdapter;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.group.DefaultChannelGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.codec.TooLongFrameException;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpObjectAggregator;
+import io.netty.handler.codec.http.HttpRequest;
+import io.netty.handler.codec.http.HttpRequestDecoder;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseEncoder;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.LastHttpContent;
+import io.netty.handler.codec.http.QueryStringDecoder;
+import io.netty.handler.ssl.SslHandler;
+import io.netty.handler.stream.ChunkedWriteHandler;
+import io.netty.handler.timeout.IdleState;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.CharsetUtil;
+import io.netty.util.concurrent.DefaultEventExecutorGroup;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DataInputByteBuffer;
@@ -79,7 +118,6 @@
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.concurrent.HadoopExecutors;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto;
import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
@@ -94,42 +132,6 @@
import org.iq80.leveldb.DB;
import org.iq80.leveldb.DBException;
import org.iq80.leveldb.Options;
-import org.jboss.netty.bootstrap.ServerBootstrap;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelFactory;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-import org.jboss.netty.channel.ChannelHandler;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.ChannelStateEvent;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.channel.group.DefaultChannelGroup;
-import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
-import org.jboss.netty.handler.codec.frame.TooLongFrameException;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
-import org.jboss.netty.handler.ssl.SslHandler;
-import org.jboss.netty.handler.stream.ChunkedWriteHandler;
-import org.jboss.netty.handler.timeout.IdleState;
-import org.jboss.netty.handler.timeout.IdleStateAwareChannelHandler;
-import org.jboss.netty.handler.timeout.IdleStateEvent;
-import org.jboss.netty.handler.timeout.IdleStateHandler;
-import org.jboss.netty.util.CharsetUtil;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timer;
import org.eclipse.jetty.http.HttpHeader;
import org.slf4j.LoggerFactory;
@@ -155,6 +157,18 @@ public class ShuffleHandler extends AuxiliaryService {
public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+
+ public static final String MAX_WEIGHT =
+ "mapreduce.shuffle.pathcache.max-weight";
+ public static final int DEFAULT_MAX_WEIGHT = 10 * 1024 * 1024;
+
+ public static final String EXPIRE_AFTER_ACCESS_MINUTES =
+ "mapreduce.shuffle.pathcache.expire-after-access-minutes";
+ public static final int DEFAULT_EXPIRE_AFTER_ACCESS_MINUTES = 5;
+
+ public static final String CONCURRENCY_LEVEL =
+ "mapreduce.shuffle.pathcache.concurrency-level";
+ public static final int DEFAULT_CONCURRENCY_LEVEL = 16;
// pattern to identify errors related to the client closing the socket early
// idea borrowed from Netty SslHandler
@@ -172,19 +186,29 @@ public class ShuffleHandler extends AuxiliaryService {
public static final HttpResponseStatus TOO_MANY_REQ_STATUS =
new HttpResponseStatus(429, "TOO MANY REQUESTS");
- // This should kept in sync with Fetcher.FETCH_RETRY_DELAY_DEFAULT
+ // This should be kept in sync with Fetcher.FETCH_RETRY_DELAY_DEFAULT
public static final long FETCH_RETRY_DELAY = 1000L;
public static final String RETRY_AFTER_HEADER = "Retry-After";
+ static final String ENCODER_HANDLER_NAME = "encoder";
private int port;
- private ChannelFactory selector;
- private final ChannelGroup accepted = new DefaultChannelGroup();
+ private EventLoopGroup bossGroup;
+ private EventLoopGroup workerGroup;
+ private ServerBootstrap bootstrap;
+ private Channel ch;
+ private final ChannelGroup accepted =
+ new DefaultChannelGroup(new DefaultEventExecutorGroup(5).next());
+ private final AtomicInteger activeConnections = new AtomicInteger();
protected HttpPipelineFactory pipelineFact;
private int sslFileBufferSize;
+
+ //TODO snemeth add a config option for these later, this is temporarily disabled for now.
+ private boolean useOutboundExceptionHandler = false;
+ private boolean useOutboundLogger = false;
/**
* Should the shuffle use posix_fadvise calls to manage the OS cache during
- * sendfile
+ * sendfile.
*/
private boolean manageOsCache;
private int readaheadLength;
@@ -194,7 +218,7 @@ public class ShuffleHandler extends AuxiliaryService {
private int maxSessionOpenFiles;
private ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
- private Map userRsrc;
+ private Map userRsrc;
private JobTokenSecretManager secretManager;
private DB stateDb = null;
@@ -225,7 +249,7 @@ public class ShuffleHandler extends AuxiliaryService {
public static final String CONNECTION_CLOSE = "close";
public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
- "mapreduce.shuffle.ssl.file.buffer.size";
+ "mapreduce.shuffle.ssl.file.buffer.size";
public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
@@ -245,7 +269,7 @@ public class ShuffleHandler extends AuxiliaryService {
public static final boolean DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED = true;
public static final boolean WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED =
false;
- private static final String TIMEOUT_HANDLER = "timeout";
+ static final String TIMEOUT_HANDLER = "timeout";
/* the maximum number of files a single GET request can
open simultaneously during shuffle
@@ -257,7 +281,6 @@ public class ShuffleHandler extends AuxiliaryService {
boolean connectionKeepAliveEnabled = false;
private int connectionKeepAliveTimeOut;
private int mapOutputMetaInfoCacheSize;
- private Timer timer;
@Metrics(about="Shuffle output metrics", context="mapred")
static class ShuffleMetrics implements ChannelFutureListener {
@@ -281,35 +304,86 @@ public void operationComplete(ChannelFuture future) throws Exception {
}
}
+ static class NettyChannelHelper {
+ static ChannelFuture writeToChannel(Channel ch, Object obj) {
+ LOG.debug("Writing {} to channel: {}", obj.getClass().getSimpleName(), ch.id());
+ return ch.writeAndFlush(obj);
+ }
+
+ static ChannelFuture writeToChannelAndClose(Channel ch, Object obj) {
+ return writeToChannel(ch, obj).addListener(ChannelFutureListener.CLOSE);
+ }
+
+ static ChannelFuture writeToChannelAndAddLastHttpContent(Channel ch, HttpResponse obj) {
+ writeToChannel(ch, obj);
+ return writeLastHttpContentToChannel(ch);
+ }
+
+ static ChannelFuture writeLastHttpContentToChannel(Channel ch) {
+ LOG.debug("Writing LastHttpContent, channel id: {}", ch.id());
+ return ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
+ }
+
+ static ChannelFuture closeChannel(Channel ch) {
+ LOG.debug("Closing channel, channel id: {}", ch.id());
+ return ch.close();
+ }
+
+ static void closeChannels(ChannelGroup channelGroup) {
+ channelGroup.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+ }
+
+ public static ChannelFuture closeAsIdle(Channel ch, int timeout) {
+ LOG.debug("Closing channel as writer was idle for {} seconds", timeout);
+ return closeChannel(ch);
+ }
+
+ public static void channelActive(Channel ch) {
+ LOG.debug("Executing channelActive, channel id: {}", ch.id());
+ }
+
+ public static void channelInactive(Channel ch) {
+ LOG.debug("Executing channelInactive, channel id: {}", ch.id());
+ }
+ }
+
+ private final MetricsSystem ms;
final ShuffleMetrics metrics;
class ReduceMapFileCount implements ChannelFutureListener {
private ReduceContext reduceContext;
- public ReduceMapFileCount(ReduceContext rc) {
+ ReduceMapFileCount(ReduceContext rc) {
this.reduceContext = rc;
}
@Override
public void operationComplete(ChannelFuture future) throws Exception {
+ LOG.trace("operationComplete");
if (!future.isSuccess()) {
- future.getChannel().close();
+ LOG.error("Future is unsuccessful. Cause: ", future.cause());
+ closeChannel(future.channel());
return;
}
int waitCount = this.reduceContext.getMapsToWait().decrementAndGet();
if (waitCount == 0) {
+ LOG.trace("Finished with all map outputs");
+ //HADOOP-15327: Need to send an instance of LastHttpContent to define HTTP
+ //message boundaries. See details in jira.
+ writeLastHttpContentToChannel(future.channel());
metrics.operationComplete(future);
// Let the idle timer handler close keep-alive connections
if (reduceContext.getKeepAlive()) {
- ChannelPipeline pipeline = future.getChannel().getPipeline();
+ ChannelPipeline pipeline = future.channel().pipeline();
TimeoutHandler timeoutHandler =
(TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
timeoutHandler.setEnabledTimeout(true);
} else {
- future.getChannel().close();
+ closeChannel(future.channel());
}
} else {
+ LOG.trace("operationComplete, waitCount > 0, invoking sendMap with reduceContext");
pipelineFact.getSHUFFLE().sendMap(reduceContext);
}
}
@@ -320,7 +394,6 @@ public void operationComplete(ChannelFuture future) throws Exception {
* Allows sendMapOutput calls from operationComplete()
*/
private static class ReduceContext {
-
private List mapIds;
private AtomicInteger mapsToWait;
private AtomicInteger mapsToSend;
@@ -331,7 +404,7 @@ private static class ReduceContext {
private String jobId;
private final boolean keepAlive;
- public ReduceContext(List mapIds, int rId,
+ ReduceContext(List mapIds, int rId,
ChannelHandlerContext context, String usr,
Map mapOutputInfoMap,
String jobId, boolean keepAlive) {
@@ -397,6 +470,7 @@ public boolean getKeepAlive() {
ShuffleHandler(MetricsSystem ms) {
super(MAPREDUCE_SHUFFLE_SERVICEID);
+ this.ms = ms;
metrics = ms.register(new ShuffleMetrics());
}
@@ -436,7 +510,8 @@ public static int deserializeMetaData(ByteBuffer meta) throws IOException {
* shuffle data requests.
* @return the serialized version of the jobToken.
*/
- public static ByteBuffer serializeServiceData(Token jobToken) throws IOException {
+ public static ByteBuffer serializeServiceData(Token jobToken)
+ throws IOException {
//TODO these bytes should be versioned
DataOutputBuffer jobToken_dob = new DataOutputBuffer();
jobToken.write(jobToken_dob);
@@ -493,6 +568,11 @@ protected void serviceInit(Configuration conf) throws Exception {
DEFAULT_MAX_SHUFFLE_CONNECTIONS);
int maxShuffleThreads = conf.getInt(MAX_SHUFFLE_THREADS,
DEFAULT_MAX_SHUFFLE_THREADS);
+ // Since Netty 4.x, the value of 0 threads would default to:
+ // io.netty.channel.MultithreadEventLoopGroup.DEFAULT_EVENT_LOOP_THREADS
+ // by simply passing 0 value to NioEventLoopGroup constructor below.
+ // However, this logic to determinte thread count
+ // was in place so we can keep it for now.
if (maxShuffleThreads == 0) {
maxShuffleThreads = 2 * Runtime.getRuntime().availableProcessors();
}
@@ -508,16 +588,14 @@ protected void serviceInit(Configuration conf) throws Exception {
DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES);
ThreadFactory bossFactory = new ThreadFactoryBuilder()
- .setNameFormat("ShuffleHandler Netty Boss #%d")
- .build();
+ .setNameFormat("ShuffleHandler Netty Boss #%d")
+ .build();
ThreadFactory workerFactory = new ThreadFactoryBuilder()
- .setNameFormat("ShuffleHandler Netty Worker #%d")
- .build();
+ .setNameFormat("ShuffleHandler Netty Worker #%d")
+ .build();
- selector = new NioServerSocketChannelFactory(
- HadoopExecutors.newCachedThreadPool(bossFactory),
- HadoopExecutors.newCachedThreadPool(workerFactory),
- maxShuffleThreads);
+ bossGroup = new NioEventLoopGroup(maxShuffleThreads, bossFactory);
+ workerGroup = new NioEventLoopGroup(maxShuffleThreads, workerFactory);
super.serviceInit(new Configuration(conf));
}
@@ -528,22 +606,24 @@ protected void serviceStart() throws Exception {
userRsrc = new ConcurrentHashMap();
secretManager = new JobTokenSecretManager();
recoverState(conf);
- ServerBootstrap bootstrap = new ServerBootstrap(selector);
- // Timer is shared across entire factory and must be released separately
- timer = new HashedWheelTimer();
try {
- pipelineFact = new HttpPipelineFactory(conf, timer);
+ pipelineFact = new HttpPipelineFactory(conf);
} catch (Exception ex) {
throw new RuntimeException(ex);
}
- bootstrap.setOption("backlog", conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE,
- DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE));
- bootstrap.setOption("child.keepAlive", true);
- bootstrap.setPipelineFactory(pipelineFact);
+
+ bootstrap = new ServerBootstrap();
+ bootstrap.group(bossGroup, workerGroup)
+ .channel(NioServerSocketChannel.class)
+ .option(ChannelOption.SO_BACKLOG,
+ conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE,
+ DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE))
+ .childOption(ChannelOption.SO_KEEPALIVE, true)
+ .childHandler(pipelineFact);
port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
- Channel ch = bootstrap.bind(new InetSocketAddress(port));
+ ch = bootstrap.bind(new InetSocketAddress(port)).sync().channel();
accepted.add(ch);
- port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+ port = ((InetSocketAddress)ch.localAddress()).getPort();
conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
pipelineFact.SHUFFLE.setPort(port);
LOG.info(getName() + " listening on port " + port);
@@ -564,21 +644,16 @@ protected void serviceStart() throws Exception {
@Override
protected void serviceStop() throws Exception {
- accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
- if (selector != null) {
- ServerBootstrap bootstrap = new ServerBootstrap(selector);
- bootstrap.releaseExternalResources();
- }
+ closeChannels(accepted);
+
if (pipelineFact != null) {
pipelineFact.destroy();
}
- if (timer != null) {
- // Release this shared timer resource
- timer.stop();
- }
+
if (stateDb != null) {
stateDb.close();
}
+ ms.unregisterSource(ShuffleMetrics.class.getSimpleName());
super.serviceStop();
}
@@ -731,7 +806,7 @@ private void recoverJobShuffleInfo(String jobIdStr, byte[] data)
JobShuffleInfoProto proto = JobShuffleInfoProto.parseFrom(data);
String user = proto.getUser();
TokenProto tokenProto = proto.getJobToken();
- Token jobToken = new Token(
+ Token jobToken = new Token<>(
tokenProto.getIdentifier().toByteArray(),
tokenProto.getPassword().toByteArray(),
new Text(tokenProto.getKind()), new Text(tokenProto.getService()));
@@ -772,29 +847,47 @@ private void removeJobShuffleInfo(JobID jobId) throws IOException {
}
}
- static class TimeoutHandler extends IdleStateAwareChannelHandler {
+ @VisibleForTesting
+ public void setUseOutboundExceptionHandler(boolean useHandler) {
+ this.useOutboundExceptionHandler = useHandler;
+ }
+ static class TimeoutHandler extends IdleStateHandler {
+ private final int connectionKeepAliveTimeOut;
private boolean enabledTimeout;
+ TimeoutHandler(int connectionKeepAliveTimeOut) {
+ //disable reader timeout
+ //set writer timeout to configured timeout value
+ //disable all idle timeout
+ super(0, connectionKeepAliveTimeOut, 0, TimeUnit.SECONDS);
+ this.connectionKeepAliveTimeOut = connectionKeepAliveTimeOut;
+ }
+
+ @VisibleForTesting
+ public int getConnectionKeepAliveTimeOut() {
+ return connectionKeepAliveTimeOut;
+ }
+
void setEnabledTimeout(boolean enabledTimeout) {
this.enabledTimeout = enabledTimeout;
}
@Override
public void channelIdle(ChannelHandlerContext ctx, IdleStateEvent e) {
- if (e.getState() == IdleState.WRITER_IDLE && enabledTimeout) {
- e.getChannel().close();
+ if (e.state() == IdleState.WRITER_IDLE && enabledTimeout) {
+ closeAsIdle(ctx.channel(), connectionKeepAliveTimeOut);
}
}
}
- class HttpPipelineFactory implements ChannelPipelineFactory {
+ class HttpPipelineFactory extends ChannelInitializer {
+ private static final int MAX_CONTENT_LENGTH = 1 << 16;
final Shuffle SHUFFLE;
private SSLFactory sslFactory;
- private final ChannelHandler idleStateHandler;
- public HttpPipelineFactory(Configuration conf, Timer timer) throws Exception {
+ HttpPipelineFactory(Configuration conf) throws Exception {
SHUFFLE = getShuffle(conf);
if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY,
MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) {
@@ -802,7 +895,6 @@ public HttpPipelineFactory(Configuration conf, Timer timer) throws Exception {
sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
sslFactory.init();
}
- this.idleStateHandler = new IdleStateHandler(timer, 0, connectionKeepAliveTimeOut, 0);
}
public Shuffle getSHUFFLE() {
@@ -815,81 +907,73 @@ public void destroy() {
}
}
- @Override
- public ChannelPipeline getPipeline() throws Exception {
- ChannelPipeline pipeline = Channels.pipeline();
+ @Override protected void initChannel(SocketChannel ch) throws Exception {
+ ChannelPipeline pipeline = ch.pipeline();
if (sslFactory != null) {
pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
}
pipeline.addLast("decoder", new HttpRequestDecoder());
- pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
- pipeline.addLast("encoder", new HttpResponseEncoder());
+ pipeline.addLast("aggregator", new HttpObjectAggregator(MAX_CONTENT_LENGTH));
+ pipeline.addLast(ENCODER_HANDLER_NAME, useOutboundLogger ?
+ new LoggingHttpResponseEncoder(false) : new HttpResponseEncoder());
pipeline.addLast("chunking", new ChunkedWriteHandler());
pipeline.addLast("shuffle", SHUFFLE);
- pipeline.addLast("idle", idleStateHandler);
- pipeline.addLast(TIMEOUT_HANDLER, new TimeoutHandler());
- return pipeline;
+ if (useOutboundExceptionHandler) {
+ //https://stackoverflow.com/questions/50612403/catch-all-exception-handling-for-outbound-channelhandler
+ pipeline.addLast("outboundExceptionHandler", new ChannelOutboundHandlerAdapter() {
+ @Override
+ public void write(ChannelHandlerContext ctx, Object msg,
+ ChannelPromise promise) throws Exception {
+ promise.addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE);
+ super.write(ctx, msg, promise);
+ }
+ });
+ }
+ pipeline.addLast(TIMEOUT_HANDLER, new TimeoutHandler(connectionKeepAliveTimeOut));
// TODO factor security manager into pipeline
// TODO factor out encode/decode to permit binary shuffle
// TODO factor out decode of index to permit alt. models
}
-
}
- class Shuffle extends SimpleChannelUpstreamHandler {
- private static final int MAX_WEIGHT = 10 * 1024 * 1024;
- private static final int EXPIRE_AFTER_ACCESS_MINUTES = 5;
- private static final int ALLOWED_CONCURRENCY = 16;
- private final Configuration conf;
+ @ChannelHandler.Sharable
+ class Shuffle extends ChannelInboundHandlerAdapter {
private final IndexCache indexCache;
+ private final LoadingCache pathCache;
+
private int port;
- private final LoadingCache pathCache =
- CacheBuilder.newBuilder().expireAfterAccess(EXPIRE_AFTER_ACCESS_MINUTES,
- TimeUnit.MINUTES).softValues().concurrencyLevel(ALLOWED_CONCURRENCY).
- removalListener(
- new RemovalListener() {
- @Override
- public void onRemoval(RemovalNotification notification) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("PathCache Eviction: " + notification.getKey() +
- ", Reason=" + notification.getCause());
- }
- }
- }
- ).maximumWeight(MAX_WEIGHT).weigher(
- new Weigher() {
- @Override
- public int weigh(AttemptPathIdentifier key,
- AttemptPathInfo value) {
- return key.jobId.length() + key.user.length() +
- key.attemptId.length()+
- value.indexPath.toString().length() +
- value.dataPath.toString().length();
- }
- }
- ).build(new CacheLoader() {
- @Override
- public AttemptPathInfo load(AttemptPathIdentifier key) throws
- Exception {
- String base = getBaseLocation(key.jobId, key.user);
- String attemptBase = base + key.attemptId;
- Path indexFileName = getAuxiliaryLocalPathHandler()
- .getLocalPathForRead(attemptBase + "/" + INDEX_FILE_NAME);
- Path mapOutputFileName = getAuxiliaryLocalPathHandler()
- .getLocalPathForRead(attemptBase + "/" + DATA_FILE_NAME);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Loaded : " + key + " via loader");
- }
- return new AttemptPathInfo(indexFileName, mapOutputFileName);
- }
- });
- public Shuffle(Configuration conf) {
- this.conf = conf;
- indexCache = new IndexCache(new JobConf(conf));
+ Shuffle(Configuration conf) {
this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
+ this.indexCache = new IndexCache(new JobConf(conf));
+ this.pathCache = CacheBuilder.newBuilder()
+ .expireAfterAccess(conf.getInt(EXPIRE_AFTER_ACCESS_MINUTES,
+ DEFAULT_EXPIRE_AFTER_ACCESS_MINUTES), TimeUnit.MINUTES)
+ .softValues()
+ .concurrencyLevel(conf.getInt(CONCURRENCY_LEVEL,
+ DEFAULT_CONCURRENCY_LEVEL))
+ .removalListener((RemovalListener) notification ->
+ LOG.debug("PathCache Eviction: {}, Reason={}",
+ notification.getKey(), notification.getCause()))
+ .maximumWeight(conf.getInt(MAX_WEIGHT, DEFAULT_MAX_WEIGHT))
+ .weigher((key, value) -> key.jobId.length() + key.user.length() +
+ key.attemptId.length()+ value.indexPath.toString().length() +
+ value.dataPath.toString().length())
+ .build(new CacheLoader() {
+ @Override
+ public AttemptPathInfo load(AttemptPathIdentifier key) throws
+ Exception {
+ String base = getBaseLocation(key.jobId, key.user);
+ String attemptBase = base + key.attemptId;
+ Path indexFileName = getAuxiliaryLocalPathHandler()
+ .getLocalPathForRead(attemptBase + "/" + INDEX_FILE_NAME);
+ Path mapOutputFileName = getAuxiliaryLocalPathHandler()
+ .getLocalPathForRead(attemptBase + "/" + DATA_FILE_NAME);
+ LOG.debug("Loaded : {} via loader", key);
+ return new AttemptPathInfo(indexFileName, mapOutputFileName);
+ }
+ });
}
public void setPort(int port) {
@@ -908,65 +992,84 @@ private List splitMaps(List mapq) {
}
@Override
- public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt)
+ public void channelActive(ChannelHandlerContext ctx)
throws Exception {
- super.channelOpen(ctx, evt);
-
- if ((maxShuffleConnections > 0) && (accepted.size() >= maxShuffleConnections)) {
+ NettyChannelHelper.channelActive(ctx.channel());
+ int numConnections = activeConnections.incrementAndGet();
+ if ((maxShuffleConnections > 0) && (numConnections > maxShuffleConnections)) {
LOG.info(String.format("Current number of shuffle connections (%d) is " +
- "greater than or equal to the max allowed shuffle connections (%d)",
+ "greater than the max allowed shuffle connections (%d)",
accepted.size(), maxShuffleConnections));
- Map headers = new HashMap(1);
+ Map headers = new HashMap<>(1);
// notify fetchers to backoff for a while before closing the connection
// if the shuffle connection limit is hit. Fetchers are expected to
// handle this notification gracefully, that is, not treating this as a
// fetch failure.
headers.put(RETRY_AFTER_HEADER, String.valueOf(FETCH_RETRY_DELAY));
sendError(ctx, "", TOO_MANY_REQ_STATUS, headers);
- return;
+ } else {
+ super.channelActive(ctx);
+ accepted.add(ctx.channel());
+ LOG.debug("Added channel: {}, channel id: {}. Accepted number of connections={}",
+ ctx.channel(), ctx.channel().id(), activeConnections.get());
}
- accepted.add(evt.getChannel());
}
@Override
- public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
+ public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+ NettyChannelHelper.channelInactive(ctx.channel());
+ super.channelInactive(ctx);
+ int noOfConnections = activeConnections.decrementAndGet();
+ LOG.debug("New value of Accepted number of connections={}", noOfConnections);
+ }
+
+ @Override
+ public void channelRead(ChannelHandlerContext ctx, Object msg)
throws Exception {
- HttpRequest request = (HttpRequest) evt.getMessage();
- if (request.getMethod() != GET) {
- sendError(ctx, METHOD_NOT_ALLOWED);
- return;
+ Channel channel = ctx.channel();
+ LOG.trace("Executing channelRead, channel id: {}", channel.id());
+ HttpRequest request = (HttpRequest) msg;
+ LOG.debug("Received HTTP request: {}, channel id: {}", request, channel.id());
+ if (request.method() != GET) {
+ sendError(ctx, METHOD_NOT_ALLOWED);
+ return;
}
// Check whether the shuffle version is compatible
- if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
- request.headers() != null ?
- request.headers().get(ShuffleHeader.HTTP_HEADER_NAME) : null)
- || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
- request.headers() != null ?
- request.headers()
- .get(ShuffleHeader.HTTP_HEADER_VERSION) : null)) {
+ String shuffleVersion = ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION;
+ String httpHeaderName = ShuffleHeader.DEFAULT_HTTP_HEADER_NAME;
+ if (request.headers() != null) {
+ shuffleVersion = request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION);
+ httpHeaderName = request.headers().get(ShuffleHeader.HTTP_HEADER_NAME);
+ LOG.debug("Received from request header: ShuffleVersion={} header name={}, channel id: {}",
+ shuffleVersion, httpHeaderName, channel.id());
+ }
+ if (request.headers() == null ||
+ !ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(httpHeaderName) ||
+ !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(shuffleVersion)) {
sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST);
}
- final Map> q =
- new QueryStringDecoder(request.getUri()).getParameters();
+ final Map> q =
+ new QueryStringDecoder(request.uri()).parameters();
final List keepAliveList = q.get("keepAlive");
boolean keepAliveParam = false;
if (keepAliveList != null && keepAliveList.size() == 1) {
keepAliveParam = Boolean.valueOf(keepAliveList.get(0));
if (LOG.isDebugEnabled()) {
- LOG.debug("KeepAliveParam : " + keepAliveList
- + " : " + keepAliveParam);
+ LOG.debug("KeepAliveParam: {} : {}, channel id: {}",
+ keepAliveList, keepAliveParam, channel.id());
}
}
final List mapIds = splitMaps(q.get("map"));
final List reduceQ = q.get("reduce");
final List jobQ = q.get("job");
if (LOG.isDebugEnabled()) {
- LOG.debug("RECV: " + request.getUri() +
+ LOG.debug("RECV: " + request.uri() +
"\n mapId: " + mapIds +
"\n reduceId: " + reduceQ +
"\n jobId: " + jobQ +
- "\n keepAlive: " + keepAliveParam);
+ "\n keepAlive: " + keepAliveParam +
+ "\n channel id: " + channel.id());
}
if (mapIds == null || reduceQ == null || jobQ == null) {
@@ -990,7 +1093,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
sendError(ctx, "Bad job parameter", BAD_REQUEST);
return;
}
- final String reqUri = request.getUri();
+ final String reqUri = request.uri();
if (null == reqUri) {
// TODO? add upstream?
sendError(ctx, FORBIDDEN);
@@ -1008,8 +1111,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
Map mapOutputInfoMap =
new HashMap();
- Channel ch = evt.getChannel();
- ChannelPipeline pipeline = ch.getPipeline();
+ ChannelPipeline pipeline = channel.pipeline();
TimeoutHandler timeoutHandler =
(TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
timeoutHandler.setEnabledTimeout(false);
@@ -1017,16 +1119,29 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
try {
populateHeaders(mapIds, jobId, user, reduceId, request,
- response, keepAliveParam, mapOutputInfoMap);
+ response, keepAliveParam, mapOutputInfoMap);
} catch(IOException e) {
- ch.write(response);
- LOG.error("Shuffle error in populating headers :", e);
- String errorMessage = getErrorMessage(e);
- sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR);
+ //HADOOP-15327
+ // Need to send an instance of LastHttpContent to define HTTP
+ // message boundaries.
+ //Sending a HTTP 200 OK + HTTP 500 later (sendError)
+ // is quite a non-standard way of crafting HTTP responses,
+ // but we need to keep backward compatibility.
+ // See more details in jira.
+ writeToChannelAndAddLastHttpContent(channel, response);
+ LOG.error("Shuffle error while populating headers. Channel id: " + channel.id(), e);
+ sendError(ctx, getErrorMessage(e), INTERNAL_SERVER_ERROR);
return;
}
- ch.write(response);
- //Initialize one ReduceContext object per messageReceived call
+ writeToChannel(channel, response).addListener((ChannelFutureListener) future -> {
+ if (future.isSuccess()) {
+ LOG.debug("Written HTTP response object successfully. Channel id: {}", channel.id());
+ } else {
+ LOG.error("Error while writing HTTP response object: {}. " +
+ "Cause: {}, channel id: {}", response, future.cause(), channel.id());
+ }
+ });
+ //Initialize one ReduceContext object per channelRead call
boolean keepAlive = keepAliveParam || connectionKeepAliveEnabled;
ReduceContext reduceContext = new ReduceContext(mapIds, reduceId, ctx,
user, mapOutputInfoMap, jobId, keepAlive);
@@ -1048,9 +1163,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
* @param reduceContext used to call sendMapOutput with correct params.
* @return the ChannelFuture of the sendMapOutput, can be null.
*/
- public ChannelFuture sendMap(ReduceContext reduceContext)
- throws Exception {
-
+ public ChannelFuture sendMap(ReduceContext reduceContext) {
+ LOG.trace("Executing sendMap");
ChannelFuture nextMap = null;
if (reduceContext.getMapsToSend().get() <
reduceContext.getMapIds().size()) {
@@ -1063,21 +1177,24 @@ public ChannelFuture sendMap(ReduceContext reduceContext)
info = getMapOutputInfo(mapId, reduceContext.getReduceId(),
reduceContext.getJobId(), reduceContext.getUser());
}
+ LOG.trace("Calling sendMapOutput");
nextMap = sendMapOutput(
reduceContext.getCtx(),
- reduceContext.getCtx().getChannel(),
+ reduceContext.getCtx().channel(),
reduceContext.getUser(), mapId,
reduceContext.getReduceId(), info);
- if (null == nextMap) {
+ if (nextMap == null) {
+ //This can only happen if spill file was not found
sendError(reduceContext.getCtx(), NOT_FOUND);
+ LOG.trace("Returning nextMap: null");
return null;
}
nextMap.addListener(new ReduceMapFileCount(reduceContext));
} catch (IOException e) {
if (e instanceof DiskChecker.DiskErrorException) {
- LOG.error("Shuffle error :" + e);
+ LOG.error("Shuffle error: " + e);
} else {
- LOG.error("Shuffle error :", e);
+ LOG.error("Shuffle error: ", e);
}
String errorMessage = getErrorMessage(e);
sendError(reduceContext.getCtx(), errorMessage,
@@ -1129,8 +1246,7 @@ protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
}
}
- IndexRecord info =
- indexCache.getIndexInformation(mapId, reduce, pathInfo.indexPath, user);
+ IndexRecord info = indexCache.getIndexInformation(mapId, reduce, pathInfo.indexPath, user);
if (LOG.isDebugEnabled()) {
LOG.debug("getMapOutputInfo: jobId=" + jobId + ", mapId=" + mapId +
@@ -1159,7 +1275,6 @@ protected void populateHeaders(List mapIds, String jobId,
outputInfo.indexRecord.rawLength, reduce);
DataOutputBuffer dob = new DataOutputBuffer();
header.write(dob);
-
contentLength += outputInfo.indexRecord.partLength;
contentLength += dob.getLength();
}
@@ -1187,14 +1302,10 @@ protected void populateHeaders(List mapIds, String jobId,
protected void setResponseHeaders(HttpResponse response,
boolean keepAliveParam, long contentLength) {
if (!connectionKeepAliveEnabled && !keepAliveParam) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Setting connection close header...");
- }
- response.headers().set(HttpHeader.CONNECTION.asString(),
- CONNECTION_CLOSE);
+ response.headers().set(HttpHeader.CONNECTION.asString(), CONNECTION_CLOSE);
} else {
response.headers().set(HttpHeader.CONTENT_LENGTH.asString(),
- String.valueOf(contentLength));
+ String.valueOf(contentLength));
response.headers().set(HttpHeader.CONNECTION.asString(),
HttpHeader.KEEP_ALIVE.asString());
response.headers().set(HttpHeader.KEEP_ALIVE.asString(),
@@ -1218,29 +1329,29 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
throws IOException {
SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid);
if (null == tokenSecret) {
- LOG.info("Request for unknown token " + appid);
- throw new IOException("could not find jobid");
+ LOG.info("Request for unknown token {}, channel id: {}", appid, ctx.channel().id());
+ throw new IOException("Could not find jobid");
}
- // string to encrypt
- String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri);
+ // encrypting URL
+ String encryptedURL = SecureShuffleUtils.buildMsgFrom(requestUri);
// hash from the fetcher
String urlHashStr =
request.headers().get(SecureShuffleUtils.HTTP_HEADER_URL_HASH);
if (urlHashStr == null) {
- LOG.info("Missing header hash for " + appid);
+ LOG.info("Missing header hash for {}, channel id: {}", appid, ctx.channel().id());
throw new IOException("fetcher cannot be authenticated");
}
if (LOG.isDebugEnabled()) {
int len = urlHashStr.length();
- LOG.debug("verifying request. enc_str=" + enc_str + "; hash=..." +
- urlHashStr.substring(len-len/2, len-1));
+ LOG.debug("Verifying request. encryptedURL:{}, hash:{}, channel id: " +
+ "{}", encryptedURL,
+ urlHashStr.substring(len - len / 2, len - 1), ctx.channel().id());
}
// verify - throws exception
- SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
+ SecureShuffleUtils.verifyReply(urlHashStr, encryptedURL, tokenSecret);
// verification passed - encode the reply
- String reply =
- SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8),
- tokenSecret);
+ String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8),
+ tokenSecret);
response.headers().set(
SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
// Put shuffle version into http header
@@ -1250,8 +1361,10 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
if (LOG.isDebugEnabled()) {
int len = reply.length();
- LOG.debug("Fetcher request verfied. enc_str=" + enc_str + ";reply=" +
- reply.substring(len-len/2, len-1));
+ LOG.debug("Fetcher request verified. " +
+ "encryptedURL: {}, reply: {}, channel id: {}",
+ encryptedURL, reply.substring(len - len / 2, len - 1),
+ ctx.channel().id());
}
}
@@ -1259,27 +1372,27 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
String user, String mapId, int reduce, MapOutputInfo mapOutputInfo)
throws IOException {
final IndexRecord info = mapOutputInfo.indexRecord;
- final ShuffleHeader header =
- new ShuffleHeader(mapId, info.partLength, info.rawLength, reduce);
+ final ShuffleHeader header = new ShuffleHeader(mapId, info.partLength, info.rawLength,
+ reduce);
final DataOutputBuffer dob = new DataOutputBuffer();
header.write(dob);
- ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ writeToChannel(ch, wrappedBuffer(dob.getData(), 0, dob.getLength()));
final File spillfile =
new File(mapOutputInfo.mapOutputFileName.toString());
RandomAccessFile spill;
try {
spill = SecureIOUtils.openForRandomRead(spillfile, "r", user, null);
} catch (FileNotFoundException e) {
- LOG.info(spillfile + " not found");
+ LOG.info("{} not found. Channel id: {}", spillfile, ctx.channel().id());
return null;
}
ChannelFuture writeFuture;
- if (ch.getPipeline().get(SslHandler.class) == null) {
+ if (ch.pipeline().get(SslHandler.class) == null) {
final FadvisedFileRegion partition = new FadvisedFileRegion(spill,
info.startOffset, info.partLength, manageOsCache, readaheadLength,
readaheadPool, spillfile.getAbsolutePath(),
shuffleBufferSize, shuffleTransferToAllowed);
- writeFuture = ch.write(partition);
+ writeFuture = writeToChannel(ch, partition);
writeFuture.addListener(new ChannelFutureListener() {
// TODO error handling; distinguish IO/connection failures,
// attribute to appropriate spill output
@@ -1288,7 +1401,7 @@ public void operationComplete(ChannelFuture future) {
if (future.isSuccess()) {
partition.transferSuccessful();
}
- partition.releaseExternalResources();
+ partition.deallocate();
}
});
} else {
@@ -1297,7 +1410,7 @@ public void operationComplete(ChannelFuture future) {
info.startOffset, info.partLength, sslFileBufferSize,
manageOsCache, readaheadLength, readaheadPool,
spillfile.getAbsolutePath());
- writeFuture = ch.write(chunk);
+ writeFuture = writeToChannel(ch, chunk);
}
metrics.shuffleConnections.incr();
metrics.shuffleOutputBytes.incr(info.partLength); // optimistic
@@ -1311,12 +1424,13 @@ protected void sendError(ChannelHandlerContext ctx,
protected void sendError(ChannelHandlerContext ctx, String message,
HttpResponseStatus status) {
- sendError(ctx, message, status, Collections.emptyMap());
+ sendError(ctx, message, status, Collections.emptyMap());
}
protected void sendError(ChannelHandlerContext ctx, String msg,
HttpResponseStatus status, Map headers) {
- HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+ FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status,
+ Unpooled.copiedBuffer(msg, CharsetUtil.UTF_8));
response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8");
// Put shuffle version into http header
response.headers().set(ShuffleHeader.HTTP_HEADER_NAME,
@@ -1326,48 +1440,45 @@ protected void sendError(ChannelHandlerContext ctx, String msg,
for (Map.Entry header : headers.entrySet()) {
response.headers().set(header.getKey(), header.getValue());
}
- response.setContent(
- ChannelBuffers.copiedBuffer(msg, CharsetUtil.UTF_8));
// Close the connection as soon as the error message is sent.
- ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+ writeToChannelAndClose(ctx.channel(), response);
}
@Override
- public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)
throws Exception {
- Channel ch = e.getChannel();
- Throwable cause = e.getCause();
+ Channel ch = ctx.channel();
if (cause instanceof TooLongFrameException) {
+ LOG.trace("TooLongFrameException, channel id: {}", ch.id());
sendError(ctx, BAD_REQUEST);
return;
} else if (cause instanceof IOException) {
if (cause instanceof ClosedChannelException) {
- LOG.debug("Ignoring closed channel error", cause);
+ LOG.debug("Ignoring closed channel error, channel id: " + ch.id(), cause);
return;
}
String message = String.valueOf(cause.getMessage());
if (IGNORABLE_ERROR_MESSAGE.matcher(message).matches()) {
- LOG.debug("Ignoring client socket close", cause);
+ LOG.debug("Ignoring client socket close, channel id: " + ch.id(), cause);
return;
}
}
- LOG.error("Shuffle error: ", cause);
- if (ch.isConnected()) {
- LOG.error("Shuffle error " + e);
+ LOG.error("Shuffle error. Channel id: " + ch.id(), cause);
+ if (ch.isActive()) {
sendError(ctx, INTERNAL_SERVER_ERROR);
}
}
}
-
+
static class AttemptPathInfo {
// TODO Change this over to just store local dir indices, instead of the
// entire path. Far more efficient.
private final Path indexPath;
private final Path dataPath;
- public AttemptPathInfo(Path indexPath, Path dataPath) {
+ AttemptPathInfo(Path indexPath, Path dataPath) {
this.indexPath = indexPath;
this.dataPath = dataPath;
}
@@ -1378,7 +1489,7 @@ static class AttemptPathIdentifier {
private final String user;
private final String attemptId;
- public AttemptPathIdentifier(String jobId, String user, String attemptId) {
+ AttemptPathIdentifier(String jobId, String user, String attemptId) {
this.jobId = jobId;
this.user = user;
this.attemptId = attemptId;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestFadvisedFileRegion.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestFadvisedFileRegion.java
index 242382e06a041..ce0c0d6aeafe9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestFadvisedFileRegion.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestFadvisedFileRegion.java
@@ -104,7 +104,7 @@ public void testCustomShuffleTransfer() throws IOException {
Assert.assertEquals(count, targetFile.length());
} finally {
if (fileRegion != null) {
- fileRegion.releaseExternalResources();
+ fileRegion.deallocate();
}
IOUtils.cleanupWithLogger(LOG, target);
IOUtils.cleanupWithLogger(LOG, targetFile);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
index 698821908e9cf..bf859e4482ea3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
@@ -17,34 +17,65 @@
*/
package org.apache.hadoop.mapred;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.DefaultFileRegion;
+import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
+import io.netty.channel.AbstractChannel;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpRequest;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseEncoder;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.timeout.IdleStateEvent;
import org.apache.hadoop.test.GenericTestUtils;
+
+import static io.netty.buffer.Unpooled.wrappedBuffer;
+import static java.util.stream.Collectors.toList;
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
-import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
-import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
import static org.junit.Assume.assumeTrue;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
+import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
+import java.io.InputStream;
import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
+import java.net.Proxy;
+import java.net.Socket;
import java.net.URL;
import java.net.SocketAddress;
+import java.net.URLConnection;
import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
import java.util.zip.CheckedOutputStream;
import java.util.zip.Checksum;
@@ -71,6 +102,7 @@
import org.apache.hadoop.service.ServiceStateException;
import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.PureJavaCrc32;
+import org.apache.hadoop.util.Sets;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -79,22 +111,13 @@
import org.apache.hadoop.yarn.server.api.AuxiliaryLocalPathHandler;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
import org.apache.hadoop.yarn.server.records.Version;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.socket.SocketChannel;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.AbstractChannel;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.hamcrest.CoreMatchers;
+import org.junit.After;
import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
+import org.junit.rules.TestName;
import org.mockito.Mockito;
import org.eclipse.jetty.http.HttpHeader;
import org.slf4j.Logger;
@@ -106,10 +129,583 @@ public class TestShuffleHandler {
LoggerFactory.getLogger(TestShuffleHandler.class);
private static final File ABS_LOG_DIR = GenericTestUtils.getTestDir(
TestShuffleHandler.class.getSimpleName() + "LocDir");
+ private static final long ATTEMPT_ID = 12345L;
+ private static final long ATTEMPT_ID_2 = 12346L;
+ private static final HttpResponseStatus OK_STATUS = new HttpResponseStatus(200, "OK");
+
+
+ //Control test execution properties with these flags
+ private static final boolean DEBUG_MODE = false;
+ //WARNING: If this is set to true and proxy server is not running, tests will fail!
+ private static final boolean USE_PROXY = false;
+ private static final int HEADER_WRITE_COUNT = 100000;
+ private static final int ARBITRARY_NEGATIVE_TIMEOUT_SECONDS = -100;
+ private static TestExecution TEST_EXECUTION;
+
+ private static class TestExecution {
+ private static final int DEFAULT_KEEP_ALIVE_TIMEOUT_SECONDS = 1;
+ private static final int DEBUG_KEEP_ALIVE_SECONDS = 1000;
+ private static final int DEFAULT_PORT = 0; //random port
+ private static final int FIXED_PORT = 8088;
+ private static final String PROXY_HOST = "127.0.0.1";
+ private static final int PROXY_PORT = 8888;
+ private static final int CONNECTION_DEBUG_TIMEOUT = 1000000;
+ private final boolean debugMode;
+ private final boolean useProxy;
+
+ TestExecution(boolean debugMode, boolean useProxy) {
+ this.debugMode = debugMode;
+ this.useProxy = useProxy;
+ }
+
+ int getKeepAliveTimeout() {
+ if (debugMode) {
+ return DEBUG_KEEP_ALIVE_SECONDS;
+ }
+ return DEFAULT_KEEP_ALIVE_TIMEOUT_SECONDS;
+ }
+
+ HttpURLConnection openConnection(URL url) throws IOException {
+ HttpURLConnection conn;
+ if (useProxy) {
+ Proxy proxy
+ = new Proxy(Proxy.Type.HTTP, new InetSocketAddress(PROXY_HOST, PROXY_PORT));
+ conn = (HttpURLConnection) url.openConnection(proxy);
+ } else {
+ conn = (HttpURLConnection) url.openConnection();
+ }
+ return conn;
+ }
+
+ int shuffleHandlerPort() {
+ if (debugMode) {
+ return FIXED_PORT;
+ } else {
+ return DEFAULT_PORT;
+ }
+ }
+
+ void parameterizeConnection(URLConnection conn) {
+ if (DEBUG_MODE) {
+ conn.setReadTimeout(CONNECTION_DEBUG_TIMEOUT);
+ conn.setConnectTimeout(CONNECTION_DEBUG_TIMEOUT);
+ }
+ }
+ }
+
+ private static class ResponseConfig {
+ private final int headerWriteCount;
+ private final int mapOutputCount;
+ private final int contentLengthOfOneMapOutput;
+ private long headerSize;
+ public long contentLengthOfResponse;
+
+ ResponseConfig(int headerWriteCount, int mapOutputCount,
+ int contentLengthOfOneMapOutput) {
+ if (mapOutputCount <= 0 && contentLengthOfOneMapOutput > 0) {
+ throw new IllegalStateException("mapOutputCount should be at least 1");
+ }
+ this.headerWriteCount = headerWriteCount;
+ this.mapOutputCount = mapOutputCount;
+ this.contentLengthOfOneMapOutput = contentLengthOfOneMapOutput;
+ }
+
+ private void setHeaderSize(long headerSize) {
+ this.headerSize = headerSize;
+ long contentLengthOfAllHeaders = headerWriteCount * headerSize;
+ this.contentLengthOfResponse = computeContentLengthOfResponse(contentLengthOfAllHeaders);
+ LOG.debug("Content-length of all headers: {}", contentLengthOfAllHeaders);
+ LOG.debug("Content-length of one MapOutput: {}", contentLengthOfOneMapOutput);
+ LOG.debug("Content-length of final HTTP response: {}", contentLengthOfResponse);
+ }
+
+ private long computeContentLengthOfResponse(long contentLengthOfAllHeaders) {
+ int mapOutputCountMultiplier = mapOutputCount;
+ if (mapOutputCount == 0) {
+ mapOutputCountMultiplier = 1;
+ }
+ return (contentLengthOfAllHeaders + contentLengthOfOneMapOutput) * mapOutputCountMultiplier;
+ }
+ }
+
+ private enum ShuffleUrlType {
+ SIMPLE, WITH_KEEPALIVE, WITH_KEEPALIVE_MULTIPLE_MAP_IDS, WITH_KEEPALIVE_NO_MAP_IDS
+ }
+
+ private static class InputStreamReadResult {
+ final String asString;
+ int totalBytesRead;
+
+ InputStreamReadResult(byte[] bytes, int totalBytesRead) {
+ this.asString = new String(bytes, StandardCharsets.UTF_8);
+ this.totalBytesRead = totalBytesRead;
+ }
+ }
+
+ private static abstract class AdditionalMapOutputSenderOperations {
+ public abstract ChannelFuture perform(ChannelHandlerContext ctx, Channel ch) throws IOException;
+ }
+
+ private class ShuffleHandlerForKeepAliveTests extends ShuffleHandler {
+ final LastSocketAddress lastSocketAddress = new LastSocketAddress();
+ final ArrayList failures = new ArrayList<>();
+ final ShuffleHeaderProvider shuffleHeaderProvider;
+ final HeaderPopulator headerPopulator;
+ MapOutputSender mapOutputSender;
+ private Consumer channelIdleCallback;
+ private CustomTimeoutHandler customTimeoutHandler;
+ private boolean failImmediatelyOnErrors = false;
+ private boolean closeChannelOnError = true;
+ private ResponseConfig responseConfig;
+
+ ShuffleHandlerForKeepAliveTests(long attemptId, ResponseConfig responseConfig,
+ Consumer channelIdleCallback) throws IOException {
+ this(attemptId, responseConfig);
+ this.channelIdleCallback = channelIdleCallback;
+ }
+
+ ShuffleHandlerForKeepAliveTests(long attemptId, ResponseConfig responseConfig)
+ throws IOException {
+ this.responseConfig = responseConfig;
+ this.shuffleHeaderProvider = new ShuffleHeaderProvider(attemptId);
+ this.responseConfig.setHeaderSize(shuffleHeaderProvider.getShuffleHeaderSize());
+ this.headerPopulator = new HeaderPopulator(this, responseConfig, shuffleHeaderProvider, true);
+ this.mapOutputSender = new MapOutputSender(responseConfig, lastSocketAddress,
+ shuffleHeaderProvider);
+ setUseOutboundExceptionHandler(true);
+ }
+
+ public void setFailImmediatelyOnErrors(boolean failImmediatelyOnErrors) {
+ this.failImmediatelyOnErrors = failImmediatelyOnErrors;
+ }
+
+ public void setCloseChannelOnError(boolean closeChannelOnError) {
+ this.closeChannelOnError = closeChannelOnError;
+ }
+
+ @Override
+ protected Shuffle getShuffle(final Configuration conf) {
+ // replace the shuffle handler with one stubbed for testing
+ return new Shuffle(conf) {
+ @Override
+ protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
+ String jobId, String user) {
+ return null;
+ }
+ @Override
+ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
+ HttpRequest request, HttpResponse response, URL requestUri) {
+ }
+
+ @Override
+ protected void populateHeaders(List mapIds, String jobId,
+ String user, int reduce, HttpRequest request,
+ HttpResponse response, boolean keepAliveParam,
+ Map infoMap) throws IOException {
+ long contentLength = headerPopulator.populateHeaders(
+ keepAliveParam);
+ super.setResponseHeaders(response, keepAliveParam, contentLength);
+ }
+
+ @Override
+ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
+ Channel ch, String user, String mapId, int reduce,
+ MapOutputInfo info) throws IOException {
+ return mapOutputSender.send(ctx, ch);
+ }
+
+ @Override
+ public void channelActive(ChannelHandlerContext ctx) throws Exception {
+ ctx.pipeline().replace(HttpResponseEncoder.class, ENCODER_HANDLER_NAME,
+ new LoggingHttpResponseEncoder(false));
+ replaceTimeoutHandlerWithCustom(ctx);
+ LOG.debug("Modified pipeline: {}", ctx.pipeline());
+ super.channelActive(ctx);
+ }
+
+ private void replaceTimeoutHandlerWithCustom(ChannelHandlerContext ctx) {
+ TimeoutHandler oldTimeoutHandler =
+ (TimeoutHandler)ctx.pipeline().get(TIMEOUT_HANDLER);
+ int timeoutValue =
+ oldTimeoutHandler.getConnectionKeepAliveTimeOut();
+ customTimeoutHandler = new CustomTimeoutHandler(timeoutValue, channelIdleCallback);
+ ctx.pipeline().replace(TIMEOUT_HANDLER, TIMEOUT_HANDLER, customTimeoutHandler);
+ }
+
+ @Override
+ protected void sendError(ChannelHandlerContext ctx,
+ HttpResponseStatus status) {
+ String message = "Error while processing request. Status: " + status;
+ handleError(ctx, message);
+ if (failImmediatelyOnErrors) {
+ stop();
+ }
+ }
+
+ @Override
+ protected void sendError(ChannelHandlerContext ctx, String message,
+ HttpResponseStatus status) {
+ String errMessage = String.format("Error while processing request. " +
+ "Status: " +
+ "%s, message: %s", status, message);
+ handleError(ctx, errMessage);
+ if (failImmediatelyOnErrors) {
+ stop();
+ }
+ }
+ };
+ }
+
+ private void handleError(ChannelHandlerContext ctx, String message) {
+ LOG.error(message);
+ failures.add(new Error(message));
+ if (closeChannelOnError) {
+ LOG.warn("sendError: Closing channel");
+ ctx.channel().close();
+ }
+ }
+
+ private class CustomTimeoutHandler extends TimeoutHandler {
+ private boolean channelIdle = false;
+ private final Consumer channelIdleCallback;
+
+ CustomTimeoutHandler(int connectionKeepAliveTimeOut,
+ Consumer channelIdleCallback) {
+ super(connectionKeepAliveTimeOut);
+ this.channelIdleCallback = channelIdleCallback;
+ }
+
+ @Override
+ public void channelIdle(ChannelHandlerContext ctx, IdleStateEvent e) {
+ LOG.debug("Channel idle");
+ this.channelIdle = true;
+ if (channelIdleCallback != null) {
+ LOG.debug("Calling channel idle callback..");
+ channelIdleCallback.accept(e);
+ }
+ super.channelIdle(ctx, e);
+ }
+ }
+ }
+
+ private static class MapOutputSender {
+ private final ResponseConfig responseConfig;
+ private final LastSocketAddress lastSocketAddress;
+ private final ShuffleHeaderProvider shuffleHeaderProvider;
+ private AdditionalMapOutputSenderOperations additionalMapOutputSenderOperations;
+
+ MapOutputSender(ResponseConfig responseConfig, LastSocketAddress lastSocketAddress,
+ ShuffleHeaderProvider shuffleHeaderProvider) {
+ this.responseConfig = responseConfig;
+ this.lastSocketAddress = lastSocketAddress;
+ this.shuffleHeaderProvider = shuffleHeaderProvider;
+ }
+
+ public ChannelFuture send(ChannelHandlerContext ctx, Channel ch) throws IOException {
+ LOG.debug("In MapOutputSender#send");
+ lastSocketAddress.setAddress(ch.remoteAddress());
+ ShuffleHeader header = shuffleHeaderProvider.createNewShuffleHeader();
+ ChannelFuture future = writeHeaderNTimes(ch, header, responseConfig.headerWriteCount);
+ // This is the last operation
+ // It's safe to increment ShuffleHeader counter for better identification
+ shuffleHeaderProvider.incrementCounter();
+ if (additionalMapOutputSenderOperations != null) {
+ return additionalMapOutputSenderOperations.perform(ctx, ch);
+ }
+ return future;
+ }
+
+ private ChannelFuture writeHeaderNTimes(Channel ch, ShuffleHeader header, int iterations)
+ throws IOException {
+ DataOutputBuffer dob = new DataOutputBuffer();
+ for (int i = 0; i < iterations; ++i) {
+ header.write(dob);
+ }
+ LOG.debug("MapOutputSender#writeHeaderNTimes WriteAndFlush big chunk of data, " +
+ "outputBufferSize: " + dob.size());
+ return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ }
+ }
+
+ private static class ShuffleHeaderProvider {
+ private final long attemptId;
+ private int attemptCounter = 0;
+ private int cachedSize = Integer.MIN_VALUE;
+
+ ShuffleHeaderProvider(long attemptId) {
+ this.attemptId = attemptId;
+ }
+
+ ShuffleHeader createNewShuffleHeader() {
+ return new ShuffleHeader(String.format("attempt_%s_1_m_1_0%s", attemptId, attemptCounter),
+ 5678, 5678, 1);
+ }
+
+ void incrementCounter() {
+ attemptCounter++;
+ }
+
+ private int getShuffleHeaderSize() throws IOException {
+ if (cachedSize != Integer.MIN_VALUE) {
+ return cachedSize;
+ }
+ DataOutputBuffer dob = new DataOutputBuffer();
+ ShuffleHeader header = createNewShuffleHeader();
+ header.write(dob);
+ cachedSize = dob.size();
+ return cachedSize;
+ }
+ }
+
+ private static class HeaderPopulator {
+ private final ShuffleHandler shuffleHandler;
+ private final boolean disableKeepAliveConfig;
+ private final ShuffleHeaderProvider shuffleHeaderProvider;
+ private final ResponseConfig responseConfig;
+
+ HeaderPopulator(ShuffleHandler shuffleHandler,
+ ResponseConfig responseConfig,
+ ShuffleHeaderProvider shuffleHeaderProvider,
+ boolean disableKeepAliveConfig) {
+ this.shuffleHandler = shuffleHandler;
+ this.responseConfig = responseConfig;
+ this.disableKeepAliveConfig = disableKeepAliveConfig;
+ this.shuffleHeaderProvider = shuffleHeaderProvider;
+ }
+
+ public long populateHeaders(boolean keepAliveParam) throws IOException {
+ // Send some dummy data (populate content length details)
+ DataOutputBuffer dob = new DataOutputBuffer();
+ for (int i = 0; i < responseConfig.headerWriteCount; ++i) {
+ ShuffleHeader header =
+ shuffleHeaderProvider.createNewShuffleHeader();
+ header.write(dob);
+ }
+ // for testing purpose;
+ // disable connectionKeepAliveEnabled if keepAliveParam is available
+ if (keepAliveParam && disableKeepAliveConfig) {
+ shuffleHandler.connectionKeepAliveEnabled = false;
+ }
+ return responseConfig.contentLengthOfResponse;
+ }
+ }
+
+ private static final class HttpConnectionData {
+ private final Map> headers;
+ private HttpURLConnection conn;
+ private final int payloadLength;
+ private final SocketAddress socket;
+ private int responseCode = -1;
+
+ private HttpConnectionData(HttpURLConnection conn, int payloadLength,
+ SocketAddress socket) {
+ this.headers = conn.getHeaderFields();
+ this.conn = conn;
+ this.payloadLength = payloadLength;
+ this.socket = socket;
+ try {
+ this.responseCode = conn.getResponseCode();
+ } catch (IOException e) {
+ fail("Failed to read response code from connection: " + conn);
+ }
+ }
+
+ static HttpConnectionData create(HttpURLConnection conn, int payloadLength,
+ SocketAddress socket) {
+ return new HttpConnectionData(conn, payloadLength, socket);
+ }
+ }
+
+ private static final class HttpConnectionAssert {
+ private final HttpConnectionData connData;
+
+ private HttpConnectionAssert(HttpConnectionData connData) {
+ this.connData = connData;
+ }
+
+ static HttpConnectionAssert create(HttpConnectionData connData) {
+ return new HttpConnectionAssert(connData);
+ }
+
+ public static void assertKeepAliveConnectionsAreSame(
+ HttpConnectionHelper httpConnectionHelper) {
+ assertTrue("At least two connection data " +
+ "is required to perform this assertion",
+ httpConnectionHelper.connectionData.size() >= 2);
+ SocketAddress firstAddress = httpConnectionHelper.getConnectionData(0).socket;
+ SocketAddress secondAddress = httpConnectionHelper.getConnectionData(1).socket;
+ Assert.assertNotNull("Initial shuffle address should not be null",
+ firstAddress);
+ Assert.assertNotNull("Keep-Alive shuffle address should not be null",
+ secondAddress);
+ assertEquals("Initial shuffle address and keep-alive shuffle "
+ + "address should be the same", firstAddress, secondAddress);
+ }
+
+ public HttpConnectionAssert expectKeepAliveWithTimeout(long timeout) {
+ assertEquals(HttpURLConnection.HTTP_OK, connData.responseCode);
+ assertHeaderValue(HttpHeader.CONNECTION, HttpHeader.KEEP_ALIVE.asString());
+ assertHeaderValue(HttpHeader.KEEP_ALIVE, "timeout=" + timeout);
+ return this;
+ }
+
+ public HttpConnectionAssert expectBadRequest(long timeout) {
+ assertEquals(HttpURLConnection.HTTP_BAD_REQUEST, connData.responseCode);
+ assertHeaderValue(HttpHeader.CONNECTION, HttpHeader.KEEP_ALIVE.asString());
+ assertHeaderValue(HttpHeader.KEEP_ALIVE, "timeout=" + timeout);
+ return this;
+ }
+
+ public HttpConnectionAssert expectResponseContentLength(long size) {
+ assertEquals(size, connData.payloadLength);
+ return this;
+ }
+
+ private void assertHeaderValue(HttpHeader header, String expectedValue) {
+ List headerList = connData.headers.get(header.asString());
+ Assert.assertNotNull("Got null header value for header: " + header, headerList);
+ Assert.assertFalse("Got empty header value for header: " + header, headerList.isEmpty());
+ assertEquals("Unexpected size of header list for header: " + header, 1,
+ headerList.size());
+ assertEquals(expectedValue, headerList.get(0));
+ }
+ }
+
+ private static class HttpConnectionHelper {
+ private final LastSocketAddress lastSocketAddress;
+ List connectionData = new ArrayList<>();
+
+ HttpConnectionHelper(LastSocketAddress lastSocketAddress) {
+ this.lastSocketAddress = lastSocketAddress;
+ }
+
+ public void connectToUrls(String[] urls, ResponseConfig responseConfig) throws IOException {
+ connectToUrlsInternal(urls, responseConfig, HttpURLConnection.HTTP_OK);
+ }
+
+ public void connectToUrls(String[] urls, ResponseConfig responseConfig, int expectedHttpStatus)
+ throws IOException {
+ connectToUrlsInternal(urls, responseConfig, expectedHttpStatus);
+ }
+
+ private void connectToUrlsInternal(String[] urls, ResponseConfig responseConfig,
+ int expectedHttpStatus) throws IOException {
+ int requests = urls.length;
+ int expectedConnections = urls.length;
+ LOG.debug("Will connect to URLs: {}", Arrays.toString(urls));
+ for (int reqIdx = 0; reqIdx < requests; reqIdx++) {
+ String urlString = urls[reqIdx];
+ LOG.debug("Connecting to URL: {}", urlString);
+ URL url = new URL(urlString);
+ HttpURLConnection conn = TEST_EXECUTION.openConnection(url);
+ conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
+ ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+ conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
+ ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+ TEST_EXECUTION.parameterizeConnection(conn);
+ conn.connect();
+ if (expectedHttpStatus == HttpURLConnection.HTTP_BAD_REQUEST) {
+ //Catch exception as error are caught with overridden sendError method
+ //Caught errors will be validated later.
+ try {
+ DataInputStream input = new DataInputStream(conn.getInputStream());
+ } catch (Exception e) {
+ expectedConnections--;
+ continue;
+ }
+ }
+ DataInputStream input = new DataInputStream(conn.getInputStream());
+ LOG.debug("Opened DataInputStream for connection: {}/{}", (reqIdx + 1), requests);
+ ShuffleHeader header = new ShuffleHeader();
+ header.readFields(input);
+ InputStreamReadResult result = readDataFromInputStream(input);
+ result.totalBytesRead += responseConfig.headerSize;
+ int expectedContentLength =
+ Integer.parseInt(conn.getHeaderField(HttpHeader.CONTENT_LENGTH.asString()));
+
+ if (result.totalBytesRead != expectedContentLength) {
+ throw new IOException(String.format("Premature EOF InputStream. " +
+ "Expected content-length: %s, " +
+ "Actual content-length: %s", expectedContentLength, result.totalBytesRead));
+ }
+ connectionData.add(HttpConnectionData
+ .create(conn, result.totalBytesRead, lastSocketAddress.getSocketAddres()));
+ input.close();
+ LOG.debug("Finished all interactions with URL: {}. Progress: {}/{}", url, (reqIdx + 1),
+ requests);
+ }
+ assertEquals(expectedConnections, connectionData.size());
+ }
+
+ void validate(Consumer connDataValidator) {
+ for (int i = 0; i < connectionData.size(); i++) {
+ LOG.debug("Validating connection data #{}", (i + 1));
+ HttpConnectionData connData = connectionData.get(i);
+ connDataValidator.accept(connData);
+ }
+ }
+
+ HttpConnectionData getConnectionData(int i) {
+ return connectionData.get(i);
+ }
+
+ private static InputStreamReadResult readDataFromInputStream(
+ InputStream input) throws IOException {
+ ByteArrayOutputStream dataStream = new ByteArrayOutputStream();
+ byte[] buffer = new byte[1024];
+ int bytesRead;
+ int totalBytesRead = 0;
+ while ((bytesRead = input.read(buffer)) != -1) {
+ dataStream.write(buffer, 0, bytesRead);
+ totalBytesRead += bytesRead;
+ }
+ LOG.debug("Read total bytes: " + totalBytesRead);
+ dataStream.flush();
+ return new InputStreamReadResult(dataStream.toByteArray(), totalBytesRead);
+ }
+ }
+
+ class ShuffleHandlerForTests extends ShuffleHandler {
+ public final ArrayList failures = new ArrayList<>();
+
+ ShuffleHandlerForTests() {
+ setUseOutboundExceptionHandler(true);
+ }
+
+ ShuffleHandlerForTests(MetricsSystem ms) {
+ super(ms);
+ setUseOutboundExceptionHandler(true);
+ }
+
+ @Override
+ protected Shuffle getShuffle(final Configuration conf) {
+ return new Shuffle(conf) {
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx,
+ Throwable cause) throws Exception {
+ LOG.debug("ExceptionCaught");
+ failures.add(cause);
+ super.exceptionCaught(ctx, cause);
+ }
+ };
+ }
+ }
class MockShuffleHandler extends org.apache.hadoop.mapred.ShuffleHandler {
- private AuxiliaryLocalPathHandler pathHandler =
+ final ArrayList failures = new ArrayList<>();
+
+ private final AuxiliaryLocalPathHandler pathHandler =
new TestAuxiliaryLocalPathHandler();
+
+ MockShuffleHandler() {
+ setUseOutboundExceptionHandler(true);
+ }
+
+ MockShuffleHandler(MetricsSystem ms) {
+ super(ms);
+ setUseOutboundExceptionHandler(true);
+ }
+
@Override
protected Shuffle getShuffle(final Configuration conf) {
return new Shuffle(conf) {
@@ -120,7 +716,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
}
@Override
protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
- String jobId, String user) throws IOException {
+ String jobId, String user) {
// Do nothing.
return null;
}
@@ -128,7 +724,7 @@ protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
protected void populateHeaders(List mapIds, String jobId,
String user, int reduce, HttpRequest request,
HttpResponse response, boolean keepAliveParam,
- Map infoMap) throws IOException {
+ Map infoMap) {
// Do nothing.
}
@Override
@@ -140,12 +736,20 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1);
DataOutputBuffer dob = new DataOutputBuffer();
header.write(dob);
- ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength()));
dob = new DataOutputBuffer();
for (int i = 0; i < 100; ++i) {
header.write(dob);
}
- return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ }
+
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx,
+ Throwable cause) throws Exception {
+ LOG.debug("ExceptionCaught");
+ failures.add(cause);
+ super.exceptionCaught(ctx, cause);
}
};
}
@@ -159,24 +763,22 @@ public AuxiliaryLocalPathHandler getAuxiliaryLocalPathHandler() {
private class TestAuxiliaryLocalPathHandler
implements AuxiliaryLocalPathHandler {
@Override
- public Path getLocalPathForRead(String path) throws IOException {
+ public Path getLocalPathForRead(String path) {
return new Path(ABS_LOG_DIR.getAbsolutePath(), path);
}
@Override
- public Path getLocalPathForWrite(String path) throws IOException {
+ public Path getLocalPathForWrite(String path) {
return new Path(ABS_LOG_DIR.getAbsolutePath());
}
@Override
- public Path getLocalPathForWrite(String path, long size)
- throws IOException {
+ public Path getLocalPathForWrite(String path, long size) {
return new Path(ABS_LOG_DIR.getAbsolutePath());
}
@Override
- public Iterable getAllLocalPathsForRead(String path)
- throws IOException {
+ public Iterable getAllLocalPathsForRead(String path) {
ArrayList paths = new ArrayList<>();
paths.add(new Path(ABS_LOG_DIR.getAbsolutePath()));
return paths;
@@ -185,16 +787,34 @@ public Iterable getAllLocalPathsForRead(String path)
private static class MockShuffleHandler2 extends
org.apache.hadoop.mapred.ShuffleHandler {
+ final ArrayList failures = new ArrayList<>(1);
boolean socketKeepAlive = false;
+
+ MockShuffleHandler2() {
+ setUseOutboundExceptionHandler(true);
+ }
+
+ MockShuffleHandler2(MetricsSystem ms) {
+ super(ms);
+ setUseOutboundExceptionHandler(true);
+ }
+
@Override
protected Shuffle getShuffle(final Configuration conf) {
return new Shuffle(conf) {
@Override
protected void verifyRequest(String appid, ChannelHandlerContext ctx,
- HttpRequest request, HttpResponse response, URL requestUri)
- throws IOException {
- SocketChannel channel = (SocketChannel)(ctx.getChannel());
- socketKeepAlive = channel.getConfig().isKeepAlive();
+ HttpRequest request, HttpResponse response, URL requestUri) {
+ SocketChannel channel = (SocketChannel)(ctx.channel());
+ socketKeepAlive = channel.config().isKeepAlive();
+ }
+
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx,
+ Throwable cause) throws Exception {
+ LOG.debug("ExceptionCaught");
+ failures.add(cause);
+ super.exceptionCaught(ctx, cause);
}
};
}
@@ -204,6 +824,38 @@ protected boolean isSocketKeepAlive() {
}
}
+ @Rule
+ public TestName name = new TestName();
+
+ @Before
+ public void setup() {
+ TEST_EXECUTION = new TestExecution(DEBUG_MODE, USE_PROXY);
+ }
+
+ @After
+ public void tearDown() {
+ int port = TEST_EXECUTION.shuffleHandlerPort();
+ if (isPortUsed(port)) {
+ String msg = String.format("Port is being used: %d. " +
+ "Current testcase name: %s",
+ port, name.getMethodName());
+ throw new IllegalStateException(msg);
+ }
+ }
+
+ private static boolean isPortUsed(int port) {
+ if (port == 0) {
+ //Don't check if port is 0
+ return false;
+ }
+ try (Socket ignored = new Socket("localhost", port)) {
+ return true;
+ } catch (IOException e) {
+ LOG.error("Port: {}, port check result: {}", port, e.getMessage());
+ return false;
+ }
+ }
+
/**
* Test the validation of ShuffleHandler's meta-data's serialization and
* de-serialization.
@@ -228,21 +880,23 @@ public void testSerializeMeta() throws Exception {
@Test (timeout = 10000)
public void testShuffleMetrics() throws Exception {
MetricsSystem ms = new MetricsSystemImpl();
- ShuffleHandler sh = new ShuffleHandler(ms);
+ ShuffleHandler sh = new ShuffleHandlerForTests(ms);
ChannelFuture cf = mock(ChannelFuture.class);
when(cf.isSuccess()).thenReturn(true).thenReturn(false);
sh.metrics.shuffleConnections.incr();
- sh.metrics.shuffleOutputBytes.incr(1*MiB);
+ sh.metrics.shuffleOutputBytes.incr(MiB);
sh.metrics.shuffleConnections.incr();
sh.metrics.shuffleOutputBytes.incr(2*MiB);
- checkShuffleMetrics(ms, 3*MiB, 0 , 0, 2);
+ checkShuffleMetrics(ms, 3*MiB, 0, 0, 2);
sh.metrics.operationComplete(cf);
sh.metrics.operationComplete(cf);
checkShuffleMetrics(ms, 3*MiB, 1, 1, 0);
+
+ sh.stop();
}
static void checkShuffleMetrics(MetricsSystem ms, long bytes, int failed,
@@ -262,57 +916,54 @@ static void checkShuffleMetrics(MetricsSystem ms, long bytes, int failed,
*/
@Test (timeout = 10000)
public void testClientClosesConnection() throws Exception {
- final ArrayList failures = new ArrayList(1);
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
- ShuffleHandler shuffleHandler = new ShuffleHandler() {
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
+ ShuffleHandlerForTests shuffleHandler = new ShuffleHandlerForTests() {
+
@Override
protected Shuffle getShuffle(Configuration conf) {
// replace the shuffle handler with one stubbed for testing
return new Shuffle(conf) {
@Override
protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
- String jobId, String user) throws IOException {
+ String jobId, String user) {
return null;
}
@Override
protected void populateHeaders(List mapIds, String jobId,
String user, int reduce, HttpRequest request,
HttpResponse response, boolean keepAliveParam,
- Map infoMap) throws IOException {
+ Map infoMap) {
// Only set response headers and skip everything else
// send some dummy value for content-length
super.setResponseHeaders(response, keepAliveParam, 100);
}
@Override
protected void verifyRequest(String appid, ChannelHandlerContext ctx,
- HttpRequest request, HttpResponse response, URL requestUri)
- throws IOException {
+ HttpRequest request, HttpResponse response, URL requestUri) {
}
@Override
protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
Channel ch, String user, String mapId, int reduce,
MapOutputInfo info)
throws IOException {
- // send a shuffle header and a lot of data down the channel
- // to trigger a broken pipe
ShuffleHeader header =
new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1);
DataOutputBuffer dob = new DataOutputBuffer();
header.write(dob);
- ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength()));
dob = new DataOutputBuffer();
for (int i = 0; i < 100000; ++i) {
header.write(dob);
}
- return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength()));
}
@Override
protected void sendError(ChannelHandlerContext ctx,
HttpResponseStatus status) {
if (failures.size() == 0) {
failures.add(new Error());
- ctx.getChannel().close();
+ ctx.channel().close();
}
}
@Override
@@ -320,7 +971,7 @@ protected void sendError(ChannelHandlerContext ctx, String message,
HttpResponseStatus status) {
if (failures.size() == 0) {
failures.add(new Error());
- ctx.getChannel().close();
+ ctx.channel().close();
}
}
};
@@ -332,26 +983,30 @@ protected void sendError(ChannelHandlerContext ctx, String message,
// simulate a reducer that closes early by reading a single shuffle header
// then closing the connection
URL url = new URL("http://127.0.0.1:"
- + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY)
- + "/mapOutput?job=job_12345_1&reduce=1&map=attempt_12345_1_m_1_0");
- HttpURLConnection conn = (HttpURLConnection)url.openConnection();
+ + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY)
+ + "/mapOutput?job=job_12345_1&reduce=1&map=attempt_12345_1_m_1_0");
+ HttpURLConnection conn = TEST_EXECUTION.openConnection(url);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
conn.connect();
DataInputStream input = new DataInputStream(conn.getInputStream());
- Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
- Assert.assertEquals("close",
+ assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+ assertEquals("close",
conn.getHeaderField(HttpHeader.CONNECTION.asString()));
ShuffleHeader header = new ShuffleHeader();
header.readFields(input);
input.close();
+ assertEquals("sendError called when client closed connection", 0,
+ shuffleHandler.failures.size());
+ assertEquals("Should have no caught exceptions", Collections.emptyList(),
+ shuffleHandler.failures);
+
shuffleHandler.stop();
- Assert.assertTrue("sendError called when client closed connection",
- failures.size() == 0);
}
+
static class LastSocketAddress {
SocketAddress lastAddress;
void setAddress(SocketAddress lastAddress) {
@@ -363,161 +1018,177 @@ SocketAddress getSocketAddres() {
}
@Test(timeout = 10000)
- public void testKeepAlive() throws Exception {
- final ArrayList failures = new ArrayList(1);
+ public void testKeepAliveInitiallyEnabled() throws Exception {
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true);
- // try setting to -ve keep alive timeout.
- conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, -100);
- final LastSocketAddress lastSocketAddress = new LastSocketAddress();
+ conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+ TEST_EXECUTION.getKeepAliveTimeout());
+ ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0);
+ ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests(
+ ATTEMPT_ID, responseConfig);
+ testKeepAliveWithHttpOk(conf, shuffleHandler, ShuffleUrlType.SIMPLE,
+ ShuffleUrlType.WITH_KEEPALIVE);
+ }
- ShuffleHandler shuffleHandler = new ShuffleHandler() {
- @Override
- protected Shuffle getShuffle(final Configuration conf) {
- // replace the shuffle handler with one stubbed for testing
- return new Shuffle(conf) {
- @Override
- protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
- String jobId, String user) throws IOException {
- return null;
- }
- @Override
- protected void verifyRequest(String appid, ChannelHandlerContext ctx,
- HttpRequest request, HttpResponse response, URL requestUri)
- throws IOException {
- }
+ @Test(timeout = 1000000)
+ public void testKeepAliveInitiallyEnabledTwoKeepAliveUrls() throws Exception {
+ Configuration conf = new Configuration();
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
+ conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true);
+ conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+ TEST_EXECUTION.getKeepAliveTimeout());
+ ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0);
+ ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests(
+ ATTEMPT_ID, responseConfig);
+ testKeepAliveWithHttpOk(conf, shuffleHandler, ShuffleUrlType.WITH_KEEPALIVE,
+ ShuffleUrlType.WITH_KEEPALIVE);
+ }
- @Override
- protected void populateHeaders(List mapIds, String jobId,
- String user, int reduce, HttpRequest request,
- HttpResponse response, boolean keepAliveParam,
- Map infoMap) throws IOException {
- // Send some dummy data (populate content length details)
- ShuffleHeader header =
- new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1);
- DataOutputBuffer dob = new DataOutputBuffer();
- header.write(dob);
- dob = new DataOutputBuffer();
- for (int i = 0; i < 100000; ++i) {
- header.write(dob);
- }
+ //TODO snemeth implement keepalive test that used properly mocked ShuffleHandler
+ @Test(timeout = 10000)
+ public void testKeepAliveInitiallyDisabled() throws Exception {
+ Configuration conf = new Configuration();
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
+ conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, false);
+ conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+ TEST_EXECUTION.getKeepAliveTimeout());
+ ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0);
+ ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests(
+ ATTEMPT_ID, responseConfig);
+ testKeepAliveWithHttpOk(conf, shuffleHandler, ShuffleUrlType.WITH_KEEPALIVE,
+ ShuffleUrlType.WITH_KEEPALIVE);
+ }
- long contentLength = dob.getLength();
- // for testing purpose;
- // disable connectinKeepAliveEnabled if keepAliveParam is available
- if (keepAliveParam) {
- connectionKeepAliveEnabled = false;
- }
+ @Test(timeout = 10000)
+ public void testKeepAliveMultipleMapAttemptIds() throws Exception {
+ final int mapOutputContentLength = 11;
+ final int mapOutputCount = 2;
- super.setResponseHeaders(response, keepAliveParam, contentLength);
+ Configuration conf = new Configuration();
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
+ conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true);
+ conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+ TEST_EXECUTION.getKeepAliveTimeout());
+ ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT,
+ mapOutputCount, mapOutputContentLength);
+ ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests(
+ ATTEMPT_ID, responseConfig);
+ shuffleHandler.mapOutputSender.additionalMapOutputSenderOperations =
+ new AdditionalMapOutputSenderOperations() {
+ @Override
+ public ChannelFuture perform(ChannelHandlerContext ctx, Channel ch) throws IOException {
+ File tmpFile = File.createTempFile("test", ".tmp");
+ Files.write(tmpFile.toPath(),
+ "dummytestcontent123456".getBytes(StandardCharsets.UTF_8));
+ final DefaultFileRegion partition = new DefaultFileRegion(tmpFile, 0,
+ mapOutputContentLength);
+ LOG.debug("Writing response partition: {}, channel: {}",
+ partition, ch.id());
+ return ch.writeAndFlush(partition)
+ .addListener((ChannelFutureListener) future ->
+ LOG.debug("Finished Writing response partition: {}, channel: " +
+ "{}", partition, ch.id()));
}
+ };
+ testKeepAliveWithHttpOk(conf, shuffleHandler,
+ ShuffleUrlType.WITH_KEEPALIVE_MULTIPLE_MAP_IDS,
+ ShuffleUrlType.WITH_KEEPALIVE_MULTIPLE_MAP_IDS);
+ }
- @Override
- protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
- Channel ch, String user, String mapId, int reduce,
- MapOutputInfo info) throws IOException {
- lastSocketAddress.setAddress(ch.getRemoteAddress());
- HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+ @Test(timeout = 10000)
+ public void testKeepAliveWithoutMapAttemptIds() throws Exception {
+ Configuration conf = new Configuration();
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
+ conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true);
+ conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+ TEST_EXECUTION.getKeepAliveTimeout());
+ ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0);
+ ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests(
+ ATTEMPT_ID, responseConfig);
+ shuffleHandler.setFailImmediatelyOnErrors(true);
+ //Closing channels caused Netty to open another channel
+ // so 1 request was handled with 2 separate channels,
+ // ultimately generating 2 * HTTP 400 errors.
+ // We'd like to avoid this so disabling closing the channel here.
+ shuffleHandler.setCloseChannelOnError(false);
+ testKeepAliveWithHttpBadRequest(conf, shuffleHandler, ShuffleUrlType.WITH_KEEPALIVE_NO_MAP_IDS);
+ }
- // send a shuffle header and a lot of data down the channel
- // to trigger a broken pipe
- ShuffleHeader header =
- new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1);
- DataOutputBuffer dob = new DataOutputBuffer();
- header.write(dob);
- ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
- dob = new DataOutputBuffer();
- for (int i = 0; i < 100000; ++i) {
- header.write(dob);
- }
- return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
- }
+ private void testKeepAliveWithHttpOk(
+ Configuration conf,
+ ShuffleHandlerForKeepAliveTests shuffleHandler,
+ ShuffleUrlType... shuffleUrlTypes) throws IOException {
+ testKeepAliveWithHttpStatus(conf, shuffleHandler, shuffleUrlTypes, HttpURLConnection.HTTP_OK);
+ }
- @Override
- protected void sendError(ChannelHandlerContext ctx,
- HttpResponseStatus status) {
- if (failures.size() == 0) {
- failures.add(new Error());
- ctx.getChannel().close();
- }
- }
+ private void testKeepAliveWithHttpBadRequest(
+ Configuration conf,
+ ShuffleHandlerForKeepAliveTests shuffleHandler,
+ ShuffleUrlType... shuffleUrlTypes) throws IOException {
+ testKeepAliveWithHttpStatus(conf, shuffleHandler, shuffleUrlTypes,
+ HttpURLConnection.HTTP_BAD_REQUEST);
+ }
- @Override
- protected void sendError(ChannelHandlerContext ctx, String message,
- HttpResponseStatus status) {
- if (failures.size() == 0) {
- failures.add(new Error());
- ctx.getChannel().close();
- }
- }
- };
- }
- };
+ private void testKeepAliveWithHttpStatus(Configuration conf,
+ ShuffleHandlerForKeepAliveTests shuffleHandler,
+ ShuffleUrlType[] shuffleUrlTypes,
+ int expectedHttpStatus) throws IOException {
+ if (expectedHttpStatus != HttpURLConnection.HTTP_BAD_REQUEST) {
+ assertTrue("Expected at least two shuffle URL types ",
+ shuffleUrlTypes.length >= 2);
+ }
shuffleHandler.init(conf);
shuffleHandler.start();
- String shuffleBaseURL = "http://127.0.0.1:"
- + shuffleHandler.getConfig().get(
- ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY);
- URL url =
- new URL(shuffleBaseURL + "/mapOutput?job=job_12345_1&reduce=1&"
- + "map=attempt_12345_1_m_1_0");
- HttpURLConnection conn = (HttpURLConnection) url.openConnection();
- conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
- ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
- conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
- ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
- conn.connect();
- DataInputStream input = new DataInputStream(conn.getInputStream());
- Assert.assertEquals(HttpHeader.KEEP_ALIVE.asString(),
- conn.getHeaderField(HttpHeader.CONNECTION.asString()));
- Assert.assertEquals("timeout=1",
- conn.getHeaderField(HttpHeader.KEEP_ALIVE.asString()));
- Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
- ShuffleHeader header = new ShuffleHeader();
- header.readFields(input);
- byte[] buffer = new byte[1024];
- while (input.read(buffer) != -1) {}
- SocketAddress firstAddress = lastSocketAddress.getSocketAddres();
- input.close();
-
- // For keepAlive via URL
- url =
- new URL(shuffleBaseURL + "/mapOutput?job=job_12345_1&reduce=1&"
- + "map=attempt_12345_1_m_1_0&keepAlive=true");
- conn = (HttpURLConnection) url.openConnection();
- conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
- ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
- conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
- ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
- conn.connect();
- input = new DataInputStream(conn.getInputStream());
- Assert.assertEquals(HttpHeader.KEEP_ALIVE.asString(),
- conn.getHeaderField(HttpHeader.CONNECTION.asString()));
- Assert.assertEquals("timeout=1",
- conn.getHeaderField(HttpHeader.KEEP_ALIVE.asString()));
- Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
- header = new ShuffleHeader();
- header.readFields(input);
- input.close();
- SocketAddress secondAddress = lastSocketAddress.getSocketAddres();
- Assert.assertNotNull("Initial shuffle address should not be null",
- firstAddress);
- Assert.assertNotNull("Keep-Alive shuffle address should not be null",
- secondAddress);
- Assert.assertEquals("Initial shuffle address and keep-alive shuffle "
- + "address should be the same", firstAddress, secondAddress);
+ String[] urls = new String[shuffleUrlTypes.length];
+ for (int i = 0; i < shuffleUrlTypes.length; i++) {
+ ShuffleUrlType url = shuffleUrlTypes[i];
+ if (url == ShuffleUrlType.SIMPLE) {
+ urls[i] = getShuffleUrl(shuffleHandler, ATTEMPT_ID, ATTEMPT_ID);
+ } else if (url == ShuffleUrlType.WITH_KEEPALIVE) {
+ urls[i] = getShuffleUrlWithKeepAlive(shuffleHandler, ATTEMPT_ID, ATTEMPT_ID);
+ } else if (url == ShuffleUrlType.WITH_KEEPALIVE_MULTIPLE_MAP_IDS) {
+ urls[i] = getShuffleUrlWithKeepAlive(shuffleHandler, ATTEMPT_ID, ATTEMPT_ID, ATTEMPT_ID_2);
+ } else if (url == ShuffleUrlType.WITH_KEEPALIVE_NO_MAP_IDS) {
+ urls[i] = getShuffleUrlWithKeepAlive(shuffleHandler, ATTEMPT_ID);
+ }
+ }
+ HttpConnectionHelper connHelper;
+ try {
+ connHelper = new HttpConnectionHelper(shuffleHandler.lastSocketAddress);
+ connHelper.connectToUrls(urls, shuffleHandler.responseConfig, expectedHttpStatus);
+ if (expectedHttpStatus == HttpURLConnection.HTTP_BAD_REQUEST) {
+ assertEquals(1, shuffleHandler.failures.size());
+ }
+ } finally {
+ shuffleHandler.stop();
+ }
+ //Verify expectations
+ int configuredTimeout = TEST_EXECUTION.getKeepAliveTimeout();
+ int expectedTimeout = configuredTimeout < 0 ? 1 : configuredTimeout;
+
+ connHelper.validate(connData -> {
+ HttpConnectionAssert.create(connData)
+ .expectKeepAliveWithTimeout(expectedTimeout)
+ .expectResponseContentLength(shuffleHandler.responseConfig.contentLengthOfResponse);
+ });
+ if (expectedHttpStatus == HttpURLConnection.HTTP_OK) {
+ HttpConnectionAssert.assertKeepAliveConnectionsAreSame(connHelper);
+ assertEquals("Unexpected ShuffleHandler failure", Collections.emptyList(),
+ shuffleHandler.failures);
+ }
}
@Test(timeout = 10000)
public void testSocketKeepAlive() throws Exception {
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true);
- // try setting to -ve keep alive timeout.
- conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, -100);
+ // try setting to negative keep alive timeout.
+ conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+ ARBITRARY_NEGATIVE_TIMEOUT_SECONDS);
HttpURLConnection conn = null;
MockShuffleHandler2 shuffleHandler = new MockShuffleHandler2();
AuxiliaryLocalPathHandler pathHandler =
@@ -535,14 +1206,16 @@ public void testSocketKeepAlive() throws Exception {
URL url =
new URL(shuffleBaseURL + "/mapOutput?job=job_12345_1&reduce=1&"
+ "map=attempt_12345_1_m_1_0");
- conn = (HttpURLConnection) url.openConnection();
+ conn = TEST_EXECUTION.openConnection(url);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
conn.connect();
+ int rc = conn.getResponseCode();
conn.getInputStream();
- Assert.assertTrue("socket should be set KEEP_ALIVE",
+ assertEquals(HttpURLConnection.HTTP_OK, rc);
+ assertTrue("socket should be set KEEP_ALIVE",
shuffleHandler.isSocketKeepAlive());
} finally {
if (conn != null) {
@@ -550,11 +1223,13 @@ public void testSocketKeepAlive() throws Exception {
}
shuffleHandler.stop();
}
+ assertEquals("Should have no caught exceptions",
+ Collections.emptyList(), shuffleHandler.failures);
}
/**
- * simulate a reducer that sends an invalid shuffle-header - sometimes a wrong
- * header_name and sometimes a wrong version
+ * Simulate a reducer that sends an invalid shuffle-header - sometimes a wrong
+ * header_name and sometimes a wrong version.
*
* @throws Exception exception
*/
@@ -562,24 +1237,24 @@ public void testSocketKeepAlive() throws Exception {
public void testIncompatibleShuffleVersion() throws Exception {
final int failureNum = 3;
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
- ShuffleHandler shuffleHandler = new ShuffleHandler();
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
+ ShuffleHandler shuffleHandler = new ShuffleHandlerForTests();
shuffleHandler.init(conf);
shuffleHandler.start();
// simulate a reducer that closes early by reading a single shuffle header
// then closing the connection
URL url = new URL("http://127.0.0.1:"
- + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY)
- + "/mapOutput?job=job_12345_1&reduce=1&map=attempt_12345_1_m_1_0");
+ + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY)
+ + "/mapOutput?job=job_12345_1&reduce=1&map=attempt_12345_1_m_1_0");
for (int i = 0; i < failureNum; ++i) {
- HttpURLConnection conn = (HttpURLConnection)url.openConnection();
+ HttpURLConnection conn = TEST_EXECUTION.openConnection(url);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
i == 0 ? "mapreduce" : "other");
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
i == 1 ? "1.0.0" : "1.0.1");
conn.connect();
- Assert.assertEquals(
+ assertEquals(
HttpURLConnection.HTTP_BAD_REQUEST, conn.getResponseCode());
}
@@ -594,10 +1269,14 @@ public void testIncompatibleShuffleVersion() throws Exception {
*/
@Test (timeout = 10000)
public void testMaxConnections() throws Exception {
+ final ArrayList failures = new ArrayList<>();
+ final int maxAllowedConnections = 3;
+ final int notAcceptedConnections = 1;
+ final int connAttempts = maxAllowedConnections + notAcceptedConnections;
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
- conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3);
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
+ conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, maxAllowedConnections);
ShuffleHandler shuffleHandler = new ShuffleHandler() {
@Override
protected Shuffle getShuffle(Configuration conf) {
@@ -605,7 +1284,7 @@ protected Shuffle getShuffle(Configuration conf) {
return new Shuffle(conf) {
@Override
protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
- String jobId, String user) throws IOException {
+ String jobId, String user) {
// Do nothing.
return null;
}
@@ -613,13 +1292,12 @@ protected MapOutputInfo getMapOutputInfo(String mapId, int reduce,
protected void populateHeaders(List mapIds, String jobId,
String user, int reduce, HttpRequest request,
HttpResponse response, boolean keepAliveParam,
- Map infoMap) throws IOException {
+ Map infoMap) {
// Do nothing.
}
@Override
protected void verifyRequest(String appid, ChannelHandlerContext ctx,
- HttpRequest request, HttpResponse response, URL requestUri)
- throws IOException {
+ HttpRequest request, HttpResponse response, URL requestUri) {
// Do nothing.
}
@Override
@@ -633,30 +1311,38 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
new ShuffleHeader("dummy_header", 5678, 5678, 1);
DataOutputBuffer dob = new DataOutputBuffer();
header.write(dob);
- ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength()));
dob = new DataOutputBuffer();
for (int i=0; i<100000; ++i) {
header.write(dob);
}
- return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ }
+
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx,
+ Throwable cause) throws Exception {
+ LOG.debug("ExceptionCaught");
+ failures.add(cause);
+ super.exceptionCaught(ctx, cause);
}
};
}
};
+ shuffleHandler.setUseOutboundExceptionHandler(true);
shuffleHandler.init(conf);
shuffleHandler.start();
// setup connections
- int connAttempts = 3;
- HttpURLConnection conns[] = new HttpURLConnection[connAttempts];
+ HttpURLConnection[] conns = new HttpURLConnection[connAttempts];
for (int i = 0; i < connAttempts; i++) {
- String URLstring = "http://127.0.0.1:"
+ String urlString = "http://127.0.0.1:"
+ shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY)
+ "/mapOutput?job=job_12345_1&reduce=1&map=attempt_12345_1_m_"
+ i + "_0";
- URL url = new URL(URLstring);
- conns[i] = (HttpURLConnection)url.openConnection();
+ URL url = new URL(urlString);
+ conns[i] = TEST_EXECUTION.openConnection(url);
conns[i].setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
conns[i].setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
@@ -668,34 +1354,61 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
conns[i].connect();
}
- //Ensure first connections are okay
- conns[0].getInputStream();
- int rc = conns[0].getResponseCode();
- Assert.assertEquals(HttpURLConnection.HTTP_OK, rc);
-
- conns[1].getInputStream();
- rc = conns[1].getResponseCode();
- Assert.assertEquals(HttpURLConnection.HTTP_OK, rc);
-
- // This connection should be closed because it to above the limit
- try {
- rc = conns[2].getResponseCode();
- Assert.assertEquals("Expected a too-many-requests response code",
- ShuffleHandler.TOO_MANY_REQ_STATUS.getCode(), rc);
- long backoff = Long.valueOf(
- conns[2].getHeaderField(ShuffleHandler.RETRY_AFTER_HEADER));
- Assert.assertTrue("The backoff value cannot be negative.", backoff > 0);
- conns[2].getInputStream();
- Assert.fail("Expected an IOException");
- } catch (IOException ioe) {
- LOG.info("Expected - connection should not be open");
- } catch (NumberFormatException ne) {
- Assert.fail("Expected a numerical value for RETRY_AFTER header field");
- } catch (Exception e) {
- Assert.fail("Expected a IOException");
+ Map> mapOfConnections = Maps.newHashMap();
+ for (HttpURLConnection conn : conns) {
+ try {
+ conn.getInputStream();
+ } catch (IOException ioe) {
+ LOG.info("Expected - connection should not be open");
+ } catch (NumberFormatException ne) {
+ fail("Expected a numerical value for RETRY_AFTER header field");
+ } catch (Exception e) {
+ fail("Expected a IOException");
+ }
+ int statusCode = conn.getResponseCode();
+ LOG.debug("Connection status code: {}", statusCode);
+ mapOfConnections.putIfAbsent(statusCode, new ArrayList<>());
+ List connectionList = mapOfConnections.get(statusCode);
+ connectionList.add(conn);
}
+
+ assertEquals(String.format("Expected only %s and %s response",
+ OK_STATUS, ShuffleHandler.TOO_MANY_REQ_STATUS),
+ Sets.newHashSet(
+ HttpURLConnection.HTTP_OK,
+ ShuffleHandler.TOO_MANY_REQ_STATUS.code()),
+ mapOfConnections.keySet());
- shuffleHandler.stop();
+ List successfulConnections =
+ mapOfConnections.get(HttpURLConnection.HTTP_OK);
+ assertEquals(String.format("Expected exactly %d requests " +
+ "with %s response", maxAllowedConnections, OK_STATUS),
+ maxAllowedConnections, successfulConnections.size());
+
+ //Ensure exactly one connection is HTTP 429 (TOO MANY REQUESTS)
+ List closedConnections =
+ mapOfConnections.get(ShuffleHandler.TOO_MANY_REQ_STATUS.code());
+ assertEquals(String.format("Expected exactly %d %s response",
+ notAcceptedConnections, ShuffleHandler.TOO_MANY_REQ_STATUS),
+ notAcceptedConnections, closedConnections.size());
+
+ // This connection should be closed because it is above the maximum limit
+ HttpURLConnection conn = closedConnections.get(0);
+ assertEquals(String.format("Expected a %s response",
+ ShuffleHandler.TOO_MANY_REQ_STATUS),
+ ShuffleHandler.TOO_MANY_REQ_STATUS.code(), conn.getResponseCode());
+ long backoff = Long.parseLong(
+ conn.getHeaderField(ShuffleHandler.RETRY_AFTER_HEADER));
+ assertTrue("The backoff value cannot be negative.", backoff > 0);
+
+ shuffleHandler.stop();
+
+ //It's okay to get a ClosedChannelException.
+ //All other kinds of exceptions means something went wrong
+ assertEquals("Should have no caught exceptions",
+ Collections.emptyList(), failures.stream()
+ .filter(f -> !(f instanceof ClosedChannelException))
+ .collect(toList()));
}
/**
@@ -706,10 +1419,11 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
*/
@Test(timeout = 100000)
public void testMapFileAccess() throws IOException {
+ final ArrayList failures = new ArrayList<>();
// This will run only in NativeIO is enabled as SecureIOUtils need it
assumeTrue(NativeIO.isAvailable());
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3);
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"kerberos");
@@ -720,7 +1434,7 @@ public void testMapFileAccess() throws IOException {
String appAttemptId = "attempt_12345_1_m_1_0";
String user = "randomUser";
String reducerId = "0";
- List fileMap = new ArrayList();
+ List fileMap = new ArrayList<>();
createShuffleHandlerFiles(ABS_LOG_DIR, user, appId.toString(), appAttemptId,
conf, fileMap);
ShuffleHandler shuffleHandler = new ShuffleHandler() {
@@ -731,15 +1445,31 @@ protected Shuffle getShuffle(Configuration conf) {
@Override
protected void verifyRequest(String appid, ChannelHandlerContext ctx,
- HttpRequest request, HttpResponse response, URL requestUri)
- throws IOException {
+ HttpRequest request, HttpResponse response, URL requestUri) {
// Do nothing.
}
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx,
+ Throwable cause) throws Exception {
+ LOG.debug("ExceptionCaught");
+ failures.add(cause);
+ super.exceptionCaught(ctx, cause);
+ }
+
+ @Override
+ public void channelActive(ChannelHandlerContext ctx) throws Exception {
+ ctx.pipeline().replace(HttpResponseEncoder.class,
+ "loggingResponseEncoder",
+ new LoggingHttpResponseEncoder(false));
+ LOG.debug("Modified pipeline: {}", ctx.pipeline());
+ super.channelActive(ctx);
+ }
};
}
};
AuxiliaryLocalPathHandler pathHandler = new TestAuxiliaryLocalPathHandler();
+ shuffleHandler.setUseOutboundExceptionHandler(true);
shuffleHandler.setAuxiliaryLocalPathHandler(pathHandler);
shuffleHandler.init(conf);
try {
@@ -747,13 +1477,13 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
DataOutputBuffer outputBuffer = new DataOutputBuffer();
outputBuffer.reset();
Token jt =
- new Token("identifier".getBytes(),
+ new Token<>("identifier".getBytes(),
"password".getBytes(), new Text(user), new Text("shuffleService"));
jt.write(outputBuffer);
shuffleHandler
- .initializeApplication(new ApplicationInitializationContext(user,
- appId, ByteBuffer.wrap(outputBuffer.getData(), 0,
- outputBuffer.getLength())));
+ .initializeApplication(new ApplicationInitializationContext(user,
+ appId, ByteBuffer.wrap(outputBuffer.getData(), 0,
+ outputBuffer.getLength())));
URL url =
new URL(
"http://127.0.0.1:"
@@ -761,32 +1491,37 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx,
ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY)
+ "/mapOutput?job=job_12345_0001&reduce=" + reducerId
+ "&map=attempt_12345_1_m_1_0");
- HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+ HttpURLConnection conn = TEST_EXECUTION.openConnection(url);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
conn.connect();
- byte[] byteArr = new byte[10000];
- try {
- DataInputStream is = new DataInputStream(conn.getInputStream());
- is.readFully(byteArr);
- } catch (EOFException e) {
- // ignore
- }
- // Retrieve file owner name
- FileInputStream is = new FileInputStream(fileMap.get(0));
- String owner = NativeIO.POSIX.getFstat(is.getFD()).getOwner();
- is.close();
+ DataInputStream is = new DataInputStream(conn.getInputStream());
+ InputStreamReadResult result = HttpConnectionHelper.readDataFromInputStream(is);
+ String receivedString = result.asString;
+
+ //Retrieve file owner name
+ FileInputStream fis = new FileInputStream(fileMap.get(0));
+ String owner = NativeIO.POSIX.getFstat(fis.getFD()).getOwner();
+ fis.close();
String message =
"Owner '" + owner + "' for path " + fileMap.get(0).getAbsolutePath()
+ " did not match expected owner '" + user + "'";
- Assert.assertTrue((new String(byteArr)).contains(message));
+ assertTrue(String.format("Received string '%s' should contain " +
+ "message '%s'", receivedString, message),
+ receivedString.contains(message));
+ assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+ LOG.info("received: " + receivedString);
+ assertNotEquals("", receivedString);
} finally {
shuffleHandler.stop();
FileUtil.fullyDelete(ABS_LOG_DIR);
}
+
+ assertEquals("Should have no caught exceptions",
+ Collections.emptyList(), failures);
}
private static void createShuffleHandlerFiles(File logDir, String user,
@@ -794,7 +1529,7 @@ private static void createShuffleHandlerFiles(File logDir, String user,
List fileMap) throws IOException {
String attemptDir =
StringUtils.join(Path.SEPARATOR,
- new String[] { logDir.getAbsolutePath(),
+ new String[] {logDir.getAbsolutePath(),
ContainerLocalizer.USERCACHE, user,
ContainerLocalizer.APPCACHE, appId, "output", appAttemptId });
File appAttemptDir = new File(attemptDir);
@@ -808,8 +1543,7 @@ private static void createShuffleHandlerFiles(File logDir, String user,
createMapOutputFile(mapOutputFile, conf);
}
- private static void
- createMapOutputFile(File mapOutputFile, Configuration conf)
+ private static void createMapOutputFile(File mapOutputFile, Configuration conf)
throws IOException {
FileOutputStream out = new FileOutputStream(mapOutputFile);
out.write("Creating new dummy map output file. Used only for testing"
@@ -846,11 +1580,11 @@ public void testRecovery() throws IOException {
final File tmpDir = new File(System.getProperty("test.build.data",
System.getProperty("java.io.tmpdir")),
TestShuffleHandler.class.getName());
- ShuffleHandler shuffle = new ShuffleHandler();
+ ShuffleHandler shuffle = new ShuffleHandlerForTests();
AuxiliaryLocalPathHandler pathHandler = new TestAuxiliaryLocalPathHandler();
shuffle.setAuxiliaryLocalPathHandler(pathHandler);
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3);
conf.set(YarnConfiguration.NM_LOCAL_DIRS,
ABS_LOG_DIR.getAbsolutePath());
@@ -861,10 +1595,10 @@ public void testRecovery() throws IOException {
shuffle.init(conf);
shuffle.start();
- // setup a shuffle token for an application
+ // set up a shuffle token for an application
DataOutputBuffer outputBuffer = new DataOutputBuffer();
outputBuffer.reset();
- Token jt = new Token(
+ Token jt = new Token<>(
"identifier".getBytes(), "password".getBytes(), new Text(user),
new Text("shuffleService"));
jt.write(outputBuffer);
@@ -874,11 +1608,11 @@ public void testRecovery() throws IOException {
// verify we are authorized to shuffle
int rc = getShuffleResponseCode(shuffle, jt);
- Assert.assertEquals(HttpURLConnection.HTTP_OK, rc);
+ assertEquals(HttpURLConnection.HTTP_OK, rc);
// emulate shuffle handler restart
shuffle.close();
- shuffle = new ShuffleHandler();
+ shuffle = new ShuffleHandlerForTests();
shuffle.setAuxiliaryLocalPathHandler(pathHandler);
shuffle.setRecoveryPath(new Path(tmpDir.toString()));
shuffle.init(conf);
@@ -886,23 +1620,23 @@ public void testRecovery() throws IOException {
// verify we are still authorized to shuffle to the old application
rc = getShuffleResponseCode(shuffle, jt);
- Assert.assertEquals(HttpURLConnection.HTTP_OK, rc);
+ assertEquals(HttpURLConnection.HTTP_OK, rc);
// shutdown app and verify access is lost
shuffle.stopApplication(new ApplicationTerminationContext(appId));
rc = getShuffleResponseCode(shuffle, jt);
- Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, rc);
+ assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, rc);
// emulate shuffle handler restart
shuffle.close();
- shuffle = new ShuffleHandler();
+ shuffle = new ShuffleHandlerForTests();
shuffle.setRecoveryPath(new Path(tmpDir.toString()));
shuffle.init(conf);
shuffle.start();
// verify we still don't have access
rc = getShuffleResponseCode(shuffle, jt);
- Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, rc);
+ assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, rc);
} finally {
if (shuffle != null) {
shuffle.close();
@@ -919,9 +1653,9 @@ public void testRecoveryFromOtherVersions() throws IOException {
System.getProperty("java.io.tmpdir")),
TestShuffleHandler.class.getName());
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3);
- ShuffleHandler shuffle = new ShuffleHandler();
+ ShuffleHandler shuffle = new ShuffleHandlerForTests();
AuxiliaryLocalPathHandler pathHandler = new TestAuxiliaryLocalPathHandler();
shuffle.setAuxiliaryLocalPathHandler(pathHandler);
conf.set(YarnConfiguration.NM_LOCAL_DIRS, ABS_LOG_DIR.getAbsolutePath());
@@ -932,10 +1666,10 @@ public void testRecoveryFromOtherVersions() throws IOException {
shuffle.init(conf);
shuffle.start();
- // setup a shuffle token for an application
+ // set up a shuffle token for an application
DataOutputBuffer outputBuffer = new DataOutputBuffer();
outputBuffer.reset();
- Token jt = new Token(
+ Token jt = new Token<>(
"identifier".getBytes(), "password".getBytes(), new Text(user),
new Text("shuffleService"));
jt.write(outputBuffer);
@@ -945,11 +1679,11 @@ public void testRecoveryFromOtherVersions() throws IOException {
// verify we are authorized to shuffle
int rc = getShuffleResponseCode(shuffle, jt);
- Assert.assertEquals(HttpURLConnection.HTTP_OK, rc);
+ assertEquals(HttpURLConnection.HTTP_OK, rc);
// emulate shuffle handler restart
shuffle.close();
- shuffle = new ShuffleHandler();
+ shuffle = new ShuffleHandlerForTests();
shuffle.setAuxiliaryLocalPathHandler(pathHandler);
shuffle.setRecoveryPath(new Path(tmpDir.toString()));
shuffle.init(conf);
@@ -957,44 +1691,44 @@ public void testRecoveryFromOtherVersions() throws IOException {
// verify we are still authorized to shuffle to the old application
rc = getShuffleResponseCode(shuffle, jt);
- Assert.assertEquals(HttpURLConnection.HTTP_OK, rc);
+ assertEquals(HttpURLConnection.HTTP_OK, rc);
Version version = Version.newInstance(1, 0);
- Assert.assertEquals(version, shuffle.getCurrentVersion());
+ assertEquals(version, shuffle.getCurrentVersion());
// emulate shuffle handler restart with compatible version
Version version11 = Version.newInstance(1, 1);
// update version info before close shuffle
shuffle.storeVersion(version11);
- Assert.assertEquals(version11, shuffle.loadVersion());
+ assertEquals(version11, shuffle.loadVersion());
shuffle.close();
- shuffle = new ShuffleHandler();
+ shuffle = new ShuffleHandlerForTests();
shuffle.setAuxiliaryLocalPathHandler(pathHandler);
shuffle.setRecoveryPath(new Path(tmpDir.toString()));
shuffle.init(conf);
shuffle.start();
// shuffle version will be override by CURRENT_VERSION_INFO after restart
// successfully.
- Assert.assertEquals(version, shuffle.loadVersion());
+ assertEquals(version, shuffle.loadVersion());
// verify we are still authorized to shuffle to the old application
rc = getShuffleResponseCode(shuffle, jt);
- Assert.assertEquals(HttpURLConnection.HTTP_OK, rc);
+ assertEquals(HttpURLConnection.HTTP_OK, rc);
// emulate shuffle handler restart with incompatible version
Version version21 = Version.newInstance(2, 1);
shuffle.storeVersion(version21);
- Assert.assertEquals(version21, shuffle.loadVersion());
+ assertEquals(version21, shuffle.loadVersion());
shuffle.close();
- shuffle = new ShuffleHandler();
+ shuffle = new ShuffleHandlerForTests();
shuffle.setAuxiliaryLocalPathHandler(pathHandler);
shuffle.setRecoveryPath(new Path(tmpDir.toString()));
shuffle.init(conf);
try {
shuffle.start();
- Assert.fail("Incompatible version, should expect fail here.");
+ fail("Incompatible version, should expect fail here.");
} catch (ServiceStateException e) {
- Assert.assertTrue("Exception message mismatch",
- e.getMessage().contains("Incompatible version for state DB schema:"));
+ assertTrue("Exception message mismatch",
+ e.getMessage().contains("Incompatible version for state DB schema:"));
}
} finally {
@@ -1010,7 +1744,7 @@ private static int getShuffleResponseCode(ShuffleHandler shuffle,
URL url = new URL("http://127.0.0.1:"
+ shuffle.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY)
+ "/mapOutput?job=job_12345_0001&reduce=0&map=attempt_12345_1_m_1_0");
- HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+ HttpURLConnection conn = TEST_EXECUTION.openConnection(url);
String encHash = SecureShuffleUtils.hashFromString(
SecureShuffleUtils.buildMsgFrom(url),
JobTokenSecretManager.createSecretKey(jt.getPassword()));
@@ -1028,9 +1762,9 @@ private static int getShuffleResponseCode(ShuffleHandler shuffle,
@Test(timeout = 100000)
public void testGetMapOutputInfo() throws Exception {
- final ArrayList failures = new ArrayList(1);
+ final ArrayList failures = new ArrayList<>(1);
Configuration conf = new Configuration();
- conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3);
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"simple");
@@ -1040,7 +1774,7 @@ public void testGetMapOutputInfo() throws Exception {
String appAttemptId = "attempt_12345_1_m_1_0";
String user = "randomUser";
String reducerId = "0";
- List fileMap = new ArrayList();
+ List fileMap = new ArrayList<>();
createShuffleHandlerFiles(ABS_LOG_DIR, user, appId.toString(), appAttemptId,
conf, fileMap);
AuxiliaryLocalPathHandler pathHandler = new TestAuxiliaryLocalPathHandler();
@@ -1062,7 +1796,7 @@ protected void populateHeaders(List mapIds,
@Override
protected void verifyRequest(String appid,
ChannelHandlerContext ctx, HttpRequest request,
- HttpResponse response, URL requestUri) throws IOException {
+ HttpResponse response, URL requestUri) {
// Do nothing.
}
@Override
@@ -1070,7 +1804,7 @@ protected void sendError(ChannelHandlerContext ctx, String message,
HttpResponseStatus status) {
if (failures.size() == 0) {
failures.add(new Error(message));
- ctx.getChannel().close();
+ ctx.channel().close();
}
}
@Override
@@ -1082,11 +1816,12 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1);
DataOutputBuffer dob = new DataOutputBuffer();
header.write(dob);
- return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+ return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength()));
}
};
}
};
+ shuffleHandler.setUseOutboundExceptionHandler(true);
shuffleHandler.setAuxiliaryLocalPathHandler(pathHandler);
shuffleHandler.init(conf);
try {
@@ -1094,8 +1829,8 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
DataOutputBuffer outputBuffer = new DataOutputBuffer();
outputBuffer.reset();
Token jt =
- new Token("identifier".getBytes(),
- "password".getBytes(), new Text(user), new Text("shuffleService"));
+ new Token<>("identifier".getBytes(),
+ "password".getBytes(), new Text(user), new Text("shuffleService"));
jt.write(outputBuffer);
shuffleHandler
.initializeApplication(new ApplicationInitializationContext(user,
@@ -1108,7 +1843,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY)
+ "/mapOutput?job=job_12345_0001&reduce=" + reducerId
+ "&map=attempt_12345_1_m_1_0");
- HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+ HttpURLConnection conn = TEST_EXECUTION.openConnection(url);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
@@ -1122,7 +1857,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
} catch (EOFException e) {
// ignore
}
- Assert.assertEquals("sendError called due to shuffle error",
+ assertEquals("sendError called due to shuffle error",
0, failures.size());
} finally {
shuffleHandler.stop();
@@ -1133,11 +1868,10 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx,
@Test(timeout = 4000)
public void testSendMapCount() throws Exception {
final List listenerList =
- new ArrayList();
-
+ new ArrayList<>();
+ int connectionKeepAliveTimeOut = 5; //arbitrary value
final ChannelHandlerContext mockCtx =
mock(ChannelHandlerContext.class);
- final MessageEvent mockEvt = mock(MessageEvent.class);
final Channel mockCh = mock(AbstractChannel.class);
final ChannelPipeline mockPipeline = mock(ChannelPipeline.class);
@@ -1146,29 +1880,23 @@ public void testSendMapCount() throws Exception {
final ChannelFuture mockFuture = createMockChannelFuture(mockCh,
listenerList);
final ShuffleHandler.TimeoutHandler timerHandler =
- new ShuffleHandler.TimeoutHandler();
+ new ShuffleHandler.TimeoutHandler(connectionKeepAliveTimeOut);
// Mock Netty Channel Context and Channel behavior
- Mockito.doReturn(mockCh).when(mockCtx).getChannel();
- when(mockCh.getPipeline()).thenReturn(mockPipeline);
+ Mockito.doReturn(mockCh).when(mockCtx).channel();
+ when(mockCh.pipeline()).thenReturn(mockPipeline);
when(mockPipeline.get(
Mockito.any(String.class))).thenReturn(timerHandler);
- when(mockCtx.getChannel()).thenReturn(mockCh);
- Mockito.doReturn(mockFuture).when(mockCh).write(Mockito.any(Object.class));
- when(mockCh.write(Object.class)).thenReturn(mockFuture);
+ when(mockCtx.channel()).thenReturn(mockCh);
+ Mockito.doReturn(mockFuture).when(mockCh).writeAndFlush(Mockito.any(Object.class));
- //Mock MessageEvent behavior
- Mockito.doReturn(mockCh).when(mockEvt).getChannel();
- when(mockEvt.getChannel()).thenReturn(mockCh);
- Mockito.doReturn(mockHttpRequest).when(mockEvt).getMessage();
-
- final ShuffleHandler sh = new MockShuffleHandler();
+ final MockShuffleHandler sh = new MockShuffleHandler();
Configuration conf = new Configuration();
sh.init(conf);
sh.start();
int maxOpenFiles =conf.getInt(ShuffleHandler.SHUFFLE_MAX_SESSION_OPEN_FILES,
ShuffleHandler.DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES);
- sh.getShuffle(conf).messageReceived(mockCtx, mockEvt);
+ sh.getShuffle(conf).channelRead(mockCtx, mockHttpRequest);
assertTrue("Number of Open files should not exceed the configured " +
"value!-Not Expected",
listenerList.size() <= maxOpenFiles);
@@ -1179,23 +1907,97 @@ public void testSendMapCount() throws Exception {
listenerList.size() <= maxOpenFiles);
}
sh.close();
+ sh.stop();
+
+ assertEquals("Should have no caught exceptions",
+ Collections.emptyList(), sh.failures);
+ }
+
+ @Test(timeout = 10000)
+ public void testIdleStateHandlingSpecifiedTimeout() throws Exception {
+ int timeoutSeconds = 4;
+ int expectedTimeoutSeconds = timeoutSeconds;
+ testHandlingIdleState(timeoutSeconds, expectedTimeoutSeconds);
+ }
+
+ @Test(timeout = 10000)
+ public void testIdleStateHandlingNegativeTimeoutDefaultsTo1Second() throws Exception {
+ int expectedTimeoutSeconds = 1; //expected by production code
+ testHandlingIdleState(ARBITRARY_NEGATIVE_TIMEOUT_SECONDS, expectedTimeoutSeconds);
+ }
+
+ private String getShuffleUrlWithKeepAlive(ShuffleHandler shuffleHandler, long jobId,
+ long... attemptIds) {
+ String url = getShuffleUrl(shuffleHandler, jobId, attemptIds);
+ return url + "&keepAlive=true";
+ }
+
+ private String getShuffleUrl(ShuffleHandler shuffleHandler, long jobId, long... attemptIds) {
+ String port = shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY);
+ String shuffleBaseURL = "http://127.0.0.1:" + port;
+
+ StringBuilder mapAttemptIds = new StringBuilder();
+ for (int i = 0; i < attemptIds.length; i++) {
+ if (i == 0) {
+ mapAttemptIds.append("&map=");
+ } else {
+ mapAttemptIds.append(",");
+ }
+ mapAttemptIds.append(String.format("attempt_%s_1_m_1_0", attemptIds[i]));
+ }
+
+ String location = String.format("/mapOutput" +
+ "?job=job_%s_1" +
+ "&reduce=1" +
+ "%s", jobId, mapAttemptIds);
+ return shuffleBaseURL + location;
+ }
+
+ private void testHandlingIdleState(int configuredTimeoutSeconds, int expectedTimeoutSeconds)
+ throws IOException,
+ InterruptedException {
+ Configuration conf = new Configuration();
+ conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, TEST_EXECUTION.shuffleHandlerPort());
+ conf.setBoolean(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, true);
+ conf.setInt(ShuffleHandler.SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT, configuredTimeoutSeconds);
+
+ final CountDownLatch countdownLatch = new CountDownLatch(1);
+ ResponseConfig responseConfig = new ResponseConfig(HEADER_WRITE_COUNT, 0, 0);
+ ShuffleHandlerForKeepAliveTests shuffleHandler = new ShuffleHandlerForKeepAliveTests(
+ ATTEMPT_ID, responseConfig,
+ event -> countdownLatch.countDown());
+ shuffleHandler.init(conf);
+ shuffleHandler.start();
+
+ String shuffleUrl = getShuffleUrl(shuffleHandler, ATTEMPT_ID, ATTEMPT_ID);
+ String[] urls = new String[] {shuffleUrl};
+ HttpConnectionHelper httpConnectionHelper = new HttpConnectionHelper(
+ shuffleHandler.lastSocketAddress);
+ long beforeConnectionTimestamp = System.currentTimeMillis();
+ httpConnectionHelper.connectToUrls(urls, shuffleHandler.responseConfig);
+ countdownLatch.await();
+ long channelClosedTimestamp = System.currentTimeMillis();
+ long secondsPassed =
+ TimeUnit.SECONDS.convert(channelClosedTimestamp - beforeConnectionTimestamp,
+ TimeUnit.MILLISECONDS);
+ assertTrue(String.format("Expected at least %s seconds of timeout. " +
+ "Actual timeout seconds: %s", expectedTimeoutSeconds, secondsPassed),
+ secondsPassed >= expectedTimeoutSeconds);
+ shuffleHandler.stop();
}
public ChannelFuture createMockChannelFuture(Channel mockCh,
final List listenerList) {
final ChannelFuture mockFuture = mock(ChannelFuture.class);
- when(mockFuture.getChannel()).thenReturn(mockCh);
+ when(mockFuture.channel()).thenReturn(mockCh);
Mockito.doReturn(true).when(mockFuture).isSuccess();
- Mockito.doAnswer(new Answer() {
- @Override
- public Object answer(InvocationOnMock invocation) throws Throwable {
- //Add ReduceMapFileCount listener to a list
- if (invocation.getArguments()[0].getClass() ==
- ShuffleHandler.ReduceMapFileCount.class)
- listenerList.add((ShuffleHandler.ReduceMapFileCount)
- invocation.getArguments()[0]);
- return null;
+ Mockito.doAnswer(invocation -> {
+ //Add ReduceMapFileCount listener to a list
+ if (invocation.getArguments()[0].getClass() == ShuffleHandler.ReduceMapFileCount.class) {
+ listenerList.add((ShuffleHandler.ReduceMapFileCount)
+ invocation.getArguments()[0]);
}
+ return null;
}).when(mockFuture).addListener(Mockito.any(
ShuffleHandler.ReduceMapFileCount.class));
return mockFuture;
@@ -1203,16 +2005,14 @@ public Object answer(InvocationOnMock invocation) throws Throwable {
public HttpRequest createMockHttpRequest() {
HttpRequest mockHttpRequest = mock(HttpRequest.class);
- Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).getMethod();
- Mockito.doAnswer(new Answer() {
- @Override
- public Object answer(InvocationOnMock invocation) throws Throwable {
- String uri = "/mapOutput?job=job_12345_1&reduce=1";
- for (int i = 0; i < 100; i++)
- uri = uri.concat("&map=attempt_12345_1_m_" + i + "_0");
- return uri;
+ Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).method();
+ Mockito.doAnswer(invocation -> {
+ String uri = "/mapOutput?job=job_12345_1&reduce=1";
+ for (int i = 0; i < 100; i++) {
+ uri = uri.concat("&map=attempt_12345_1_m_" + i + "_0");
}
- }).when(mockHttpRequest).getUri();
+ return uri;
+ }).when(mockHttpRequest).uri();
return mockHttpRequest;
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/log4j.properties b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/log4j.properties
index 81a3f6ad5d248..b7d8ad36efc26 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/log4j.properties
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/resources/log4j.properties
@@ -17,3 +17,5 @@ log4j.threshold=ALL
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n
+log4j.logger.io.netty=INFO
+log4j.logger.org.apache.hadoop.mapred=INFO
\ No newline at end of file
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
index 40bea6501c6ca..b447ddb008312 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
@@ -41,8 +41,8 @@
protobuf-java
- org.apache.avro
- avro
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-avro_1_11org.eclipse.jetty
@@ -54,7 +54,7 @@
org.jboss.netty
- netty
+ netty-allorg.apache.velocity
@@ -68,6 +68,10 @@
paranamer-antcom.thoughtworks.paranamer
+
+ org.xerial.snappy
+ snappy-java
+
@@ -120,13 +124,18 @@
io.netty
- netty
+ netty-allcommons-loggingcommons-loggingprovided
+
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-guava
+ provided
+ com.google.guavaguava
@@ -153,7 +162,7 @@
test
-
+
@@ -310,7 +319,7 @@
-
+
hadoop-mapreduce-client-corehadoop-mapreduce-client-common
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
index 03f8d0e6851fd..b36a5de486dbb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
@@ -117,7 +117,7 @@
slf4j-api
-
+
diff --git a/hadoop-mapreduce-project/pom.xml b/hadoop-mapreduce-project/pom.xml
index 6f15d70ed35b7..d0e3bd1ce11c1 100644
--- a/hadoop-mapreduce-project/pom.xml
+++ b/hadoop-mapreduce-project/pom.xml
@@ -49,6 +49,7 @@
org.apache.avroavro
+ ${avro.version}org.eclipse.jetty
@@ -60,7 +61,7 @@
io.netty
- netty
+ netty-allorg.apache.velocity
@@ -76,6 +77,25 @@
+
+ com.google.protobuf
+ protobuf-java
+
+
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-avro_1_11
+ ${hadoop-thirdparty.avro.version}
+
+
+ org.slf4j
+ slf4j-api
+
+
+ org.xerial.snappy
+ snappy-java
+
+
+ org.apache.hadoophadoop-common
@@ -132,7 +152,7 @@
io.netty
- netty
+ netty-allcommons-io
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 35ccaabe6e7bb..49cf5361a5142 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -35,7 +35,7 @@
falsetrue
- 9.4.40.v20210413
+ 9.4.43.v20210629__
@@ -61,16 +61,15 @@
${basedir}/../../hadoop-common-project/hadoop-common/targetfile:///dev/urandom
-
- 1.7.7
+
+ 1.11.3
- 1.19
+ 1.19.4
- 1.9.13
- 2.10.5
- 2.10.5.1
+ 2.13.2
+ 2.13.2.24.5.13
@@ -88,12 +87,20 @@
2.5.0${env.HADOOP_PROTOC_PATH}
+ 1.2.0
+ ${hadoop-thirdparty.version}
+ ${hadoop-thirdparty.version}
+ ${hadoop-thirdparty.version}
+ org.apache.hadoop.thirdparty
+ ${hadoop-thirdparty-shaded-prefix}.protobuf
+ ${hadoop-thirdparty-shaded-prefix}.com.google.common
+
3.5.10.3.3.0.0-SNAPSHOT4.2.03.0.52.1.7
- 27.0-jre
+ 32.0.1-jre4.02.9.9
@@ -106,7 +113,7 @@
1.9.41.2
- 1.11
+ 1.143.2.21.211.0
@@ -117,6 +124,7 @@
3.1.13.61.10.0
+ 1.1.10.40.3.0-eca3531-SNAPSHOT
@@ -127,12 +135,12 @@
2.7.51.12.2.21
- 2.2.4
+ 2.9.03.1.0-incubating4.1.0-incubating3.2.4
- 3.10.6.Final
- 4.1.68.Final
+ 4.1.94.Final
+ 1.1.10.40.5.1
@@ -180,12 +188,12 @@
${hadoop.version}1.5.4
- 1.26
+ 2.01.4.82.0.24.13.23.12.2
- 5.3.0
+ 5.4.02.4.79.8.11.10.11
@@ -209,6 +217,11 @@
jdiff${jdiff.version}
+
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-guava
+ ${hadoop-thirdparty-guava.version}
+ org.apache.hadoophadoop-assemblies
@@ -266,6 +279,18 @@
org.slf4jslf4j-reload4j
+
+ org.codehaus.jackson
+ jackson-mapper-asl
+
+
+ org.codehaus.jackson
+ jackson-jaxrs
+
+
+ org.codehaus.jackson
+ jackson-xc
+
@@ -789,13 +814,33 @@
${jersey.version}
- com.sun.jersey
+ com.github.pjfanningjersey-json
- ${jersey.version}
+ 1.20
- stax
- stax-api
+ com.fasterxml.jackson.core
+ jackson-core
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+
+
+ org.codehaus.jackson
+ jackson-core-asl
+
+
+ org.codehaus.jackson
+ jackson-mapper-asl
+
+
+ org.codehaus.jackson
+ jackson-jaxrs
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-json-provider
@@ -886,13 +931,67 @@
io.netty
- netty
- ${netty3.version}
+ netty-all
+ ${netty4.version}io.netty
- netty-all
+ netty-codec-socks
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-handler-proxy
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-resolver
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-handler
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-buffer
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-transport
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-common
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-transport-native-unix-common
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-transport-native-epoll
+ ${netty4.version}
+
+
+
+ io.netty
+ netty-codec${netty4.version}
@@ -1060,26 +1159,6 @@
woodstox-core${woodstox.version}
-
- org.codehaus.jackson
- jackson-mapper-asl
- ${jackson.version}
-
-
- org.codehaus.jackson
- jackson-core-asl
- ${jackson.version}
-
-
- org.codehaus.jackson
- jackson-jaxrs
- ${jackson.version}
-
-
- org.codehaus.jackson
- jackson-xc
- ${jackson.version}
- com.fasterxml.jackson.corejackson-core
@@ -1108,7 +1187,7 @@
org.mockitomockito-all
- 1.8.5
+ 1.10.19org.objenesis
@@ -1135,9 +1214,9 @@
- org.apache.avro
- avro
- ${avro.version}
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-avro_1_11
+ ${hadoop-thirdparty.avro.version}net.sf.kosmosfs
@@ -1210,7 +1289,7 @@
org.jboss.netty
- netty
+ netty-alljline
@@ -1238,7 +1317,7 @@
org.jboss.netty
- netty
+ netty-alljline
@@ -1331,7 +1410,7 @@
org.codehaus.jettisonjettison
- 1.1
+ 1.5.4stax
@@ -1993,16 +2072,16 @@
cglib:cglib:3.2.0com.google.inject:guice:4.0
- com.sun.jersey:jersey-core:1.19
- com.sun.jersey:jersey-servlet:1.19
- com.sun.jersey:jersey-json:1.19
- com.sun.jersey:jersey-server:1.19
- com.sun.jersey:jersey-client:1.19
- com.sun.jersey:jersey-grizzly2:1.19
- com.sun.jersey:jersey-grizzly2-servlet:1.19
- com.sun.jersey.jersey-test-framework:jersey-test-framework-core:1.19
- com.sun.jersey.jersey-test-framework:jersey-test-framework-grizzly2:1.19
- com.sun.jersey.contribs:jersey-guice:1.19
+ com.sun.jersey:jersey-core:1.19.4
+ com.sun.jersey:jersey-servlet:1.19.4
+ com.github.pjfanning:jersey-json:1.20
+ com.sun.jersey:jersey-server:1.19.4
+ com.sun.jersey:jersey-client:1.19.4
+ com.sun.jersey:jersey-grizzly2:1.19.4
+ com.sun.jersey:jersey-grizzly2-servlet:1.19.4
+ com.sun.jersey.jersey-test-framework:jersey-test-framework-core:1.19.4
+ com.sun.jersey.jersey-test-framework:jersey-test-framework-grizzly2:1.19.4
+ com.sun.jersey.contribs:jersey-guice:1.19.4org.ow2.asm:asm:5.0.0
@@ -2236,5 +2315,6 @@
+
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
index 4c5c1fdb8add9..972b8dbef8ea5 100644
--- a/hadoop-tools/hadoop-azure/pom.xml
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -63,12 +63,12 @@
org.apache.maven.pluginsmaven-checkstyle-plugin
-
src/config/checkstyle.xmlsrc/config/checkstyle-suppressions.xml
-
+
@@ -103,7 +103,7 @@
-
+
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/pom.xml
index acea229f7e41f..0fcf0ce18fc84 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/pom.xml
@@ -155,6 +155,10 @@
org.apache.hadoophadoop-hdfs-client
+
+ javax.ws.rs
+ javax.ws.rs-api
+ org.apache.hadoophadoop-client
diff --git a/pom.xml b/pom.xml
index b9abef4b69585..c465fda7bdb1f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -47,7 +47,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x
-
${distMgmtSnapshotsId}
@@ -247,7 +246,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x
org.apache.avroavro-maven-plugin
- [1.5.3,)
+ [1.9.2,)schemaprotocol