Browse Source

YARN-2938. Fixed new findbugs warnings in hadoop-yarn-resourcemanager and hadoop-yarn-applicationhistoryservice. Contributed by Varun Saxena.

(cherry picked from commit 241d3b3a50c6af92f023d8b2c24598f4813f4674)
Zhijie Shen 10 years ago
parent
commit
e5cb52ec1f

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

@@ -260,6 +260,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2992. ZKRMStateStore crashes due to session expiry. (Karthik Kambatla
     via jianhe)
 
+    YARN-2938. Fixed new findbugs warnings in hadoop-yarn-resourcemanager and
+    hadoop-yarn-applicationhistoryservice. (Varun Saxena via zjshen)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 32 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml

@@ -417,4 +417,36 @@
     <Class name="org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.sharedcache.SharedCacheUploadService" />
     <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
   </Match>
+
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt" />
+    <Field name="queue" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler" />
+    <Field name="allocConf" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode" />
+    <Field name="numContainers" />
+    <Bug pattern="VO_VOLATILE_INCREMENT" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue" />
+    <Field name="numContainers" />
+    <Bug pattern="VO_VOLATILE_INCREMENT" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue" />
+    <Field name="numApplications" />
+    <Bug pattern="VO_VOLATILE_INCREMENT" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler" />
+    <Method name="recoverContainersOnNode" />
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE" />
+  </Match>
+
 </FindBugsFilter>

+ 39 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java

@@ -48,6 +48,7 @@ import org.iq80.leveldb.*;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.*;
 import java.util.Map.Entry;
 import java.util.concurrent.locks.ReentrantLock;
@@ -120,25 +121,25 @@ public class LeveldbTimelineStore extends AbstractService
   @VisibleForTesting
   static final String FILENAME = "leveldb-timeline-store.ldb";
 
-  private static final byte[] START_TIME_LOOKUP_PREFIX = "k".getBytes();
-  private static final byte[] ENTITY_ENTRY_PREFIX = "e".getBytes();
-  private static final byte[] INDEXED_ENTRY_PREFIX = "i".getBytes();
+  private static final byte[] START_TIME_LOOKUP_PREFIX = "k".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] ENTITY_ENTRY_PREFIX = "e".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] INDEXED_ENTRY_PREFIX = "i".getBytes(Charset.forName("UTF-8"));
 
-  private static final byte[] EVENTS_COLUMN = "e".getBytes();
-  private static final byte[] PRIMARY_FILTERS_COLUMN = "f".getBytes();
-  private static final byte[] OTHER_INFO_COLUMN = "i".getBytes();
-  private static final byte[] RELATED_ENTITIES_COLUMN = "r".getBytes();
+  private static final byte[] EVENTS_COLUMN = "e".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] PRIMARY_FILTERS_COLUMN = "f".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] OTHER_INFO_COLUMN = "i".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] RELATED_ENTITIES_COLUMN = "r".getBytes(Charset.forName("UTF-8"));
   private static final byte[] INVISIBLE_REVERSE_RELATED_ENTITIES_COLUMN =
-      "z".getBytes();
-  private static final byte[] DOMAIN_ID_COLUMN = "d".getBytes();
+      "z".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] DOMAIN_ID_COLUMN = "d".getBytes(Charset.forName("UTF-8"));
 
-  private static final byte[] DOMAIN_ENTRY_PREFIX = "d".getBytes();
-  private static final byte[] OWNER_LOOKUP_PREFIX = "o".getBytes();
-  private static final byte[] DESCRIPTION_COLUMN = "d".getBytes();
-  private static final byte[] OWNER_COLUMN = "o".getBytes();
-  private static final byte[] READER_COLUMN = "r".getBytes();
-  private static final byte[] WRITER_COLUMN = "w".getBytes();
-  private static final byte[] TIMESTAMP_COLUMN = "t".getBytes();
+  private static final byte[] DOMAIN_ENTRY_PREFIX = "d".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] OWNER_LOOKUP_PREFIX = "o".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] DESCRIPTION_COLUMN = "d".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] OWNER_COLUMN = "o".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] READER_COLUMN = "r".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] WRITER_COLUMN = "w".getBytes(Charset.forName("UTF-8"));
+  private static final byte[] TIMESTAMP_COLUMN = "t".getBytes(Charset.forName("UTF-8"));
 
   private static final byte[] EMPTY_BYTES = new byte[0];
   
@@ -440,7 +441,7 @@ public class LeveldbTimelineStore extends AbstractService
         }
       } else if (key[prefixlen] == DOMAIN_ID_COLUMN[0]) {
         byte[] v = iterator.peekNext().getValue();
-        String domainId = new String(v);
+        String domainId = new String(v, Charset.forName("UTF-8"));
         entity.setDomainId(domainId);
       } else {
         if (key[prefixlen] !=
@@ -790,7 +791,7 @@ public class LeveldbTimelineStore extends AbstractService
               if (domainIdBytes == null) {
                 domainId = TimelineDataManager.DEFAULT_DOMAIN_ID;
               } else {
-                domainId = new String(domainIdBytes);
+                domainId = new String(domainIdBytes, Charset.forName("UTF-8"));
               }
               if (!domainId.equals(entity.getDomainId())) {
                 // in this case the entity will be put, but the relation will be
@@ -845,9 +846,9 @@ public class LeveldbTimelineStore extends AbstractService
           return;
         }
       } else {
-        writeBatch.put(key, entity.getDomainId().getBytes());
+        writeBatch.put(key, entity.getDomainId().getBytes(Charset.forName("UTF-8")));
         writePrimaryFilterEntries(writeBatch, primaryFilters, key,
-            entity.getDomainId().getBytes());
+            entity.getDomainId().getBytes(Charset.forName("UTF-8")));
       }
       db.write(writeBatch);
     } catch (DBException de) {
@@ -879,7 +880,7 @@ public class LeveldbTimelineStore extends AbstractService
           // This is the new entity, the domain should be the same
         byte[] key = createDomainIdKey(relatedEntity.getId(),
             relatedEntity.getType(), relatedEntityStartTime);
-        db.put(key, entity.getDomainId().getBytes());
+        db.put(key, entity.getDomainId().getBytes(Charset.forName("UTF-8")));
         db.put(createRelatedEntityKey(relatedEntity.getId(),
             relatedEntity.getType(), relatedEntityStartTime,
             entity.getEntityId(), entity.getEntityType()), EMPTY_BYTES);
@@ -1206,7 +1207,7 @@ public class LeveldbTimelineStore extends AbstractService
    * to the end of the array (for parsing other info keys).
    */
   private static String parseRemainingKey(byte[] b, int offset) {
-    return new String(b, offset, b.length - offset);
+    return new String(b, offset, b.length - offset, Charset.forName("UTF-8"));
   }
 
   /**
@@ -1589,8 +1590,10 @@ public class LeveldbTimelineStore extends AbstractService
       byte[] ownerLookupEntryKey = createOwnerLookupKey(
           domain.getOwner(), domain.getId(), DESCRIPTION_COLUMN);
       if (domain.getDescription() != null) {
-        writeBatch.put(domainEntryKey, domain.getDescription().getBytes());
-        writeBatch.put(ownerLookupEntryKey, domain.getDescription().getBytes());
+        writeBatch.put(domainEntryKey, domain.getDescription().
+                       getBytes(Charset.forName("UTF-8")));
+        writeBatch.put(ownerLookupEntryKey, domain.getDescription().
+                       getBytes(Charset.forName("UTF-8")));
       } else {
         writeBatch.put(domainEntryKey, EMPTY_BYTES);
         writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES);
@@ -1601,16 +1604,17 @@ public class LeveldbTimelineStore extends AbstractService
       ownerLookupEntryKey = createOwnerLookupKey(
           domain.getOwner(), domain.getId(), OWNER_COLUMN);
       // Null check for owner is done before
-      writeBatch.put(domainEntryKey, domain.getOwner().getBytes());
-      writeBatch.put(ownerLookupEntryKey, domain.getOwner().getBytes());
+      writeBatch.put(domainEntryKey, domain.getOwner().getBytes(Charset.forName("UTF-8")));
+      writeBatch.put(ownerLookupEntryKey, domain.getOwner().getBytes(Charset.forName("UTF-8")));
 
       // Write readers
       domainEntryKey = createDomainEntryKey(domain.getId(), READER_COLUMN);
       ownerLookupEntryKey = createOwnerLookupKey(
           domain.getOwner(), domain.getId(), READER_COLUMN);
       if (domain.getReaders() != null && domain.getReaders().length() > 0) {
-        writeBatch.put(domainEntryKey, domain.getReaders().getBytes());
-        writeBatch.put(ownerLookupEntryKey, domain.getReaders().getBytes());
+        writeBatch.put(domainEntryKey, domain.getReaders().getBytes(Charset.forName("UTF-8")));
+        writeBatch.put(ownerLookupEntryKey, domain.getReaders().
+                       getBytes(Charset.forName("UTF-8")));
       } else {
         writeBatch.put(domainEntryKey, EMPTY_BYTES);
         writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES);
@@ -1621,8 +1625,9 @@ public class LeveldbTimelineStore extends AbstractService
       ownerLookupEntryKey = createOwnerLookupKey(
           domain.getOwner(), domain.getId(), WRITER_COLUMN);
       if (domain.getWriters() != null && domain.getWriters().length() > 0) {
-        writeBatch.put(domainEntryKey, domain.getWriters().getBytes());
-        writeBatch.put(ownerLookupEntryKey, domain.getWriters().getBytes());
+        writeBatch.put(domainEntryKey, domain.getWriters().getBytes(Charset.forName("UTF-8")));
+        writeBatch.put(ownerLookupEntryKey, domain.getWriters().
+                       getBytes(Charset.forName("UTF-8")));
       } else {
         writeBatch.put(domainEntryKey, EMPTY_BYTES);
         writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES);
@@ -1759,13 +1764,13 @@ public class LeveldbTimelineStore extends AbstractService
       byte[] value = iterator.peekNext().getValue();
       if (value != null && value.length > 0) {
         if (key[prefix.length] == DESCRIPTION_COLUMN[0]) {
-          domain.setDescription(new String(value));
+          domain.setDescription(new String(value, Charset.forName("UTF-8")));
         } else if (key[prefix.length] == OWNER_COLUMN[0]) {
-          domain.setOwner(new String(value));
+          domain.setOwner(new String(value, Charset.forName("UTF-8")));
         } else if (key[prefix.length] == READER_COLUMN[0]) {
-          domain.setReaders(new String(value));
+          domain.setReaders(new String(value, Charset.forName("UTF-8")));
         } else if (key[prefix.length] == WRITER_COLUMN[0]) {
-          domain.setWriters(new String(value));
+          domain.setWriters(new String(value, Charset.forName("UTF-8")));
         } else if (key[prefix.length] == TIMESTAMP_COLUMN[0]) {
           domain.setCreatedTime(readReverseOrderedLong(value, 0));
           domain.setModifiedTime(readReverseOrderedLong(value, 8));

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java

@@ -398,7 +398,7 @@ public class MemoryTimelineStore
   public void put(TimelineDomain domain) throws IOException {
     TimelineDomain domainToReplace =
         domainsById.get(domain.getId());
-    long currentTimestamp = System.currentTimeMillis();
+    Long currentTimestamp = System.currentTimeMillis();
     TimelineDomain domainToStore = createTimelineDomain(
         domain.getId(), domain.getDescription(), domain.getOwner(),
         domain.getReaders(), domain.getWriters(),

+ 7 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java

@@ -18,9 +18,12 @@
 
 package org.apache.hadoop.yarn.server.timeline.security;
 
-import java.io.FileReader;
+import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.Reader;
+import java.nio.charset.Charset;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -113,7 +116,9 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
       Reader reader = null;
       try {
         StringBuilder secret = new StringBuilder();
-        reader = new FileReader(signatureSecretFile);
+        reader = new InputStreamReader(new FileInputStream(new File(signatureSecretFile)),
+                                      Charset.forName("UTF-8"));
+
         int c = reader.read();
         while (c > -1) {
           secret.append((char) c);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java

@@ -23,6 +23,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.security.NoSuchAlgorithmException;
 import java.security.SecureRandom;
 import java.util.ArrayList;
@@ -1108,7 +1109,7 @@ public class ZKRMStateStore extends RMStateStore {
         }
         if (useDefaultFencingScheme) {
           zkClient.addAuthInfo(zkRootNodeAuthScheme,
-              (zkRootNodeUsername + ":" + zkRootNodePassword).getBytes());
+              (zkRootNodeUsername + ":" + zkRootNodePassword).getBytes(Charset.forName("UTF-8")));
         }
       } catch (IOException ioe) {
         // Retry in case of network failures

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java

@@ -448,6 +448,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     case UNHEALTHY:
       metrics.decrNumUnhealthyNMs();
       break;
+    default:
+      LOG.debug("Unexpected previous node state");    
     }
   }
 
@@ -462,6 +464,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       case UNHEALTHY:
         metrics.decrNumUnhealthyNMs();
         break;
+      default:
+        LOG.debug("Unexpected inital state");
     }
 
     switch (finalState) {
@@ -477,6 +481,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     case UNHEALTHY:
       metrics.incrNumUnhealthyNMs();
       break;
+    default:
+      LOG.debug("Unexpected final state");
     }
   }
 
@@ -583,6 +589,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
             case UNHEALTHY:
               ClusterMetrics.getMetrics().decrNumUnhealthyNMs();
               break;
+            default:
+              LOG.debug("Unexpected Rmnode state");
             }
             rmNode.context.getRMNodes().put(newNode.getNodeID(), newNode);
             rmNode.context.getDispatcher().getEventHandler().handle(

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java

@@ -48,10 +48,10 @@ public abstract class SchedulingPolicy {
    * Returns a {@link SchedulingPolicy} instance corresponding to the passed clazz
    */
   public static SchedulingPolicy getInstance(Class<? extends SchedulingPolicy> clazz) {
-    SchedulingPolicy policy = instances.get(clazz);
-    if (policy == null) {
-      policy = ReflectionUtils.newInstance(clazz, null);
-      instances.put(clazz, policy);
+    SchedulingPolicy policy = ReflectionUtils.newInstance(clazz, null);
+    SchedulingPolicy policyRet = instances.putIfAbsent(clazz, policy);
+    if(policyRet != null) {
+      return policyRet;
     }
     return policy;
   }

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java

@@ -92,6 +92,8 @@ class NodesPage extends RmView {
           rmNodes = this.rm.getRMContext().getInactiveRMNodes().values();
           isInactive = true;
           break;
+        default:
+          LOG.debug("Unexpected state filter for inactive RM node");
         }
       }
       for (RMNode ni : rmNodes) {