Kaynağa Gözat

YARN-321. Forwarding YARN-321 branch to latest trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/YARN-321@1561207 13f79535-47bb-0310-9956-ffa450edef68
Vinod Kumar Vavilapalli 11 yıl önce
ebeveyn
işleme
f72176cf3f

+ 6 - 3
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -424,6 +424,9 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10143 replace WritableFactories's hashmap with ConcurrentHashMap
     (Liang Xie via stack)
 
+    HADOOP-9652. Allow RawLocalFs#getFileLinkStatus to fill in the link owner
+    and mode if requested. (Andrew Wang via Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -450,9 +453,6 @@ Release 2.4.0 - UNRELEASED
     HADOOP-9817. FileSystem#globStatus and FileContext#globStatus need to work
     with symlinks. (Colin Patrick McCabe via Andrew Wang)
 
-    HADOOP-9652.  RawLocalFs#getFileLinkStatus does not fill in the link owner
-    and mode.  (Andrew Wang via Colin Patrick McCabe)
-
     HADOOP-9875.  TestDoAsEffectiveUser can fail on JDK 7.  (Aaron T. Myers via
     Colin Patrick McCabe)
 
@@ -556,6 +556,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10132. RPC#stopProxy() should log the class of proxy when IllegalArgumentException 
     is encountered (Ted yu via umamahesh)
 
+    HADOOP-10248. Property name should be included in the exception where property value 
+    is null (Akira AJISAKA via umamahesh)
+
   OPTIMIZATIONS
 
     HADOOP-10142. Avoid groups lookup for unprivileged users such as "dr.who"

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -963,7 +963,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
         "Property name must not be null");
     Preconditions.checkArgument(
         value != null,
-        "Property value must not be null");
+        "The value of property " + name + " must not be null");
     DeprecationContext deprecations = deprecationContext.get();
     if (deprecations.getDeprecatedKeyMap().isEmpty()) {
       getProps();

+ 10 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -16,8 +16,11 @@
  * limitations under the License.
  */
 
+
 package org.apache.hadoop.fs;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.BufferedOutputStream;
 import java.io.DataOutput;
 import java.io.File;
@@ -51,7 +54,13 @@ import org.apache.hadoop.util.StringUtils;
 public class RawLocalFileSystem extends FileSystem {
   static final URI NAME = URI.create("file:///");
   private Path workingDir;
-  private static final boolean useDeprecatedFileStatus = !Stat.isAvailable();
+  // Temporary workaround for HADOOP-9652.
+  private static boolean useDeprecatedFileStatus = true;
+
+  @VisibleForTesting
+  public static void useStatIfAvailable() {
+    useDeprecatedFileStatus = !Stat.isAvailable();
+  }
   
   public RawLocalFileSystem() {
     workingDir = getInitialWorkingDirectory();

+ 3 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -1183,6 +1183,8 @@ public class TestConfiguration extends TestCase {
       fail("Should throw an IllegalArgumentException exception ");
     } catch (Exception e) {
       assertTrue(e instanceof IllegalArgumentException);
+      assertEquals(e.getMessage(),
+          "The value of property testClassName must not be null");
     }
   }
 
@@ -1193,6 +1195,7 @@ public class TestConfiguration extends TestCase {
       fail("Should throw an IllegalArgumentException exception ");
     } catch (Exception e) {
       assertTrue(e instanceof IllegalArgumentException);
+      assertEquals(e.getMessage(), "Property name must not be null");
     }
   }
 

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java

@@ -38,6 +38,11 @@ import org.junit.Test;
  * Test symbolic links using LocalFs.
  */
 abstract public class TestSymlinkLocalFS extends SymlinkBaseTest {
+
+  // Workaround for HADOOP-9652
+  static {
+    RawLocalFileSystem.useStatIfAvailable();
+  }
   
   @Override
   protected String getScheme() {

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -456,6 +456,9 @@ Release 2.4.0 - UNRELEASED
     YARN-1607. TestRM relies on the scheduler assigning multiple containers in
     a single node update (Sandy Ryza)
 
+    YARN-1575. Public localizer crashes with "Localized unkown resource"
+    (jlowe)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 8 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java

@@ -638,8 +638,8 @@ public class ResourceLocalizationService extends CompositeService
       super("Public Localizer");
       this.lfs = getLocalFileContext(conf);
       this.conf = conf;
-      this.pending =
-          new ConcurrentHashMap<Future<Path>, LocalizerResourceRequestEvent>();
+      this.pending = Collections.synchronizedMap(
+          new HashMap<Future<Path>, LocalizerResourceRequestEvent>());
       this.threadPool = createLocalizerExecutor(conf);
       this.queue = new ExecutorCompletionService<Path>(threadPool);
     }
@@ -675,8 +675,12 @@ public class ResourceLocalizationService extends CompositeService
             publicDirDestPath =
                 new Path(publicDirDestPath, Long.toString(publicRsrc
                   .nextUniqueNumber()));
-            pending.put(queue.submit(new FSDownload(lfs, null, conf,
-              publicDirDestPath, resource)), request);
+            // explicitly synchronize pending here to avoid future task
+            // completing and being dequeued before pending updated
+            synchronized (pending) {
+              pending.put(queue.submit(new FSDownload(lfs, null, conf,
+                  publicDirDestPath, resource)), request);
+            }
           } catch (IOException e) {
             rsrc.unlock();
             // TODO Need to Fix IO Exceptions - Notifying resource