Prechádzať zdrojové kódy

YARN-9958. Remove the invalid lock in ContainerExecutor (#1704)

Wanqiang Ji 5 rokov pred
rodič
commit
c48de9aa2d

+ 4 - 28
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java

@@ -34,8 +34,6 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.slf4j.Logger;
@@ -101,8 +99,6 @@ public abstract class ContainerExecutor implements Configurable {
   private final ConcurrentMap<ContainerId, Path> pidFiles =
       new ConcurrentHashMap<>();
   private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-  private final ReadLock readLock = lock.readLock();
-  private final WriteLock writeLock = lock.writeLock();
   private String[] whitelistVars;
 
   @Override
@@ -570,12 +566,7 @@ public abstract class ContainerExecutor implements Configurable {
    * @return the path of the pid-file for the given containerId.
    */
   protected Path getPidFilePath(ContainerId containerId) {
-    readLock.lock();
-    try {
-      return (this.pidFiles.get(containerId));
-    } finally {
-      readLock.unlock();
-    }
+    return this.pidFiles.get(containerId);
   }
 
   /**
@@ -721,12 +712,7 @@ public abstract class ContainerExecutor implements Configurable {
    * @return true if the container is active
    */
   protected boolean isContainerActive(ContainerId containerId) {
-    readLock.lock();
-    try {
-      return (this.pidFiles.containsKey(containerId));
-    } finally {
-      readLock.unlock();
-    }
+    return this.pidFiles.containsKey(containerId);
   }
 
   @VisibleForTesting
@@ -742,12 +728,7 @@ public abstract class ContainerExecutor implements Configurable {
    * of the launched process
    */
   public void activateContainer(ContainerId containerId, Path pidFilePath) {
-    writeLock.lock();
-    try {
-      this.pidFiles.put(containerId, pidFilePath);
-    } finally {
-      writeLock.unlock();
-    }
+    this.pidFiles.put(containerId, pidFilePath);
   }
 
   // LinuxContainerExecutor overrides this method and behaves differently.
@@ -778,12 +759,7 @@ public abstract class ContainerExecutor implements Configurable {
    * @param containerId the container ID
    */
   public void deactivateContainer(ContainerId containerId) {
-    writeLock.lock();
-    try {
-      this.pidFiles.remove(containerId);
-    } finally {
-      writeLock.unlock();
-    }
+    this.pidFiles.remove(containerId);
   }
 
   /**