Skip to content

Commit 8c84525

Browse files
committed
MAPREDUCE-7522. Fix SpotBugs issues in hadoop-mapreduce-client-app module.
1 parent 516cf46 commit 8c84525

File tree

11 files changed

+49
-30
lines changed

11 files changed

+49
-30
lines changed

hadoop-mapreduce-project/dev-support/findbugs-exclude.xml

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -554,4 +554,21 @@
554554
<Method name="onSuccess" />
555555
<Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE" />
556556
</Match>
557+
558+
<Match>
559+
<Class name="org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl" />
560+
<Bug pattern="CT_CONSTRUCTOR_THROW" />
561+
</Match>
562+
<Match>
563+
<Class name="org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator" />
564+
<Bug pattern="CT_CONSTRUCTOR_THROW" />
565+
</Match>
566+
<Match>
567+
<Class name="org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo" />
568+
<Bug pattern="CT_CONSTRUCTOR_THROW" />
569+
</Match>
570+
<Match>
571+
<Class name="org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo" />
572+
<Bug pattern="CT_CONSTRUCTOR_THROW" />
573+
</Match>
557574
</FindBugsFilter>

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -209,7 +209,7 @@ public class MRAppMaster extends CompositeService {
209209
protected JobTokenSecretManager jobTokenSecretManager =
210210
new JobTokenSecretManager();
211211
private JobId jobId;
212-
private boolean newApiCommitter;
212+
private volatile boolean newApiCommitter;
213213
private ClassLoader jobClassLoader;
214214
private OutputCommitter committer;
215215
private JobEventDispatcher jobEventDispatcher;

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ public boolean isReported(){
7777
//received from a task.
7878
private Thread lostTaskCheckerThread;
7979
private volatile boolean stopped;
80-
private long taskTimeOut;
80+
private volatile long taskTimeOut;
8181
private long unregisterTimeOut;
8282
private long taskStuckTimeOut;
8383
private int taskTimeOutCheckInterval = 30 * 1000; // 30 seconds.

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/commit/CommitterEventHandler.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -338,7 +338,7 @@ private synchronized void waitForValidCommitWindow()
338338
@Override
339339
public void run() {
340340
synchronized (EventProcessor.this) {
341-
EventProcessor.this.notify();
341+
EventProcessor.this.notifyAll();
342342
}
343343
}
344344
});

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -633,15 +633,15 @@ JobEventType.JOB_KILL, new KillTasksTransition())
633633
//changing fields while the job is running
634634
private int numMapTasks;
635635
private int numReduceTasks;
636-
private int completedTaskCount = 0;
637-
private int succeededMapTaskCount = 0;
636+
private volatile int completedTaskCount = 0;
637+
private volatile int succeededMapTaskCount = 0;
638638
private int succeededReduceTaskCount = 0;
639639
private int failedMapTaskCount = 0;
640640
private int failedReduceTaskCount = 0;
641641
private int killedMapTaskCount = 0;
642642
private int killedReduceTaskCount = 0;
643643
private long startTime;
644-
private long finishTime;
644+
private volatile long finishTime;
645645
private float setupProgress;
646646
private float mapProgress;
647647
private float reduceProgress;

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -190,16 +190,16 @@ public abstract class TaskAttemptImpl implements
190190
private static Object commonContainerSpecLock = new Object();
191191
private static ContainerLaunchContext commonContainerSpec = null;
192192
private static final Object classpathLock = new Object();
193-
private long launchTime;
194-
private long finishTime;
193+
private volatile long launchTime;
194+
private volatile long finishTime;
195195
private WrappedProgressSplitsBlock progressSplitBlock;
196-
private int shufflePort = -1;
196+
private volatile int shufflePort = -1;
197197
private String trackerName;
198-
private int httpPort;
198+
private volatile int httpPort;
199199
private Locality locality;
200200
private Avataar avataar;
201201
private boolean rescheduleNextAttempt = false;
202-
private boolean failFast = false;
202+
private volatile boolean failFast = false;
203203

204204
private static final CleanupContainerTransition
205205
CLEANUP_CONTAINER_TRANSITION = new CleanupContainerTransition();
@@ -648,7 +648,7 @@ TaskAttemptEventType.TA_CONTAINER_CLEANED, new TaskCleanupTransition())
648648
stateMachine;
649649

650650
@VisibleForTesting
651-
public Container container;
651+
private Container container;
652652
private String nodeRackName;
653653
private WrappedJvmID jvmID;
654654

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,7 @@ public class ContainerLauncherImpl extends AbstractService implements
7474
new ConcurrentHashMap<ContainerId, Container>();
7575
private final AppContext context;
7676
protected ThreadPoolExecutor launcherPool;
77-
protected int initialPoolSize;
77+
protected volatile int initialPoolSize;
7878
private int limitOnPoolSize;
7979
private Thread eventHandlingThread;
8080
protected BlockingQueue<ContainerLauncherEvent> eventQueue =

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,7 @@ public abstract class RMCommunicator extends AbstractService
6969
implements RMHeartbeatHandler {
7070
private static final Logger LOG =
7171
LoggerFactory.getLogger(RMCommunicator.class);
72-
private int rmPollInterval;//millis
72+
private volatile int rmPollInterval;//millis
7373
protected ApplicationId applicationId;
7474
private final AtomicBoolean stopped;
7575
protected Thread allocatorThread;
@@ -90,7 +90,7 @@ public abstract class RMCommunicator extends AbstractService
9090
// Has a signal (SIGTERM etc) been issued?
9191
protected volatile boolean isSignalled = false;
9292
private volatile boolean shouldUnregister = true;
93-
private boolean isApplicationMasterRegistered = false;
93+
private volatile boolean isApplicationMasterRegistered = false;
9494

9595
private EnumSet<SchedulerResourceTypes> schedulerResourceTypes;
9696

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

Lines changed: 9 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -165,15 +165,15 @@ added to the pending and are ramped up (added to scheduled) based
165165
private Resource mapResourceRequest = Resources.none();
166166
private Resource reduceResourceRequest = Resources.none();
167167

168-
private boolean reduceStarted = false;
168+
private volatile boolean reduceStarted = false;
169169
private float maxReduceRampupLimit = 0;
170-
private float maxReducePreemptionLimit = 0;
170+
private volatile float maxReducePreemptionLimit = 0;
171171

172172
// Mapper allocation timeout, after which a reducer is forcibly preempted
173-
private long reducerUnconditionalPreemptionDelayMs;
173+
private volatile long reducerUnconditionalPreemptionDelayMs;
174174

175175
// Duration to wait before preempting a reducer when there is NO room
176-
private long reducerNoHeadroomPreemptionDelayMs = 0;
176+
private volatile long reducerNoHeadroomPreemptionDelayMs = 0;
177177

178178
private float reduceSlowStart = 0;
179179
private int maxRunningMaps = 0;
@@ -1292,9 +1292,11 @@ private void containerAssigned(Container allocated,
12921292
}
12931293

12941294
private void containerNotAssigned(Container allocated) {
1295-
containersReleased++;
1296-
pendingRelease.add(allocated.getId());
1297-
release(allocated.getId());
1295+
synchronized (RMContainerAllocator.this) {
1296+
containersReleased++;
1297+
pendingRelease.add(allocated.getId());
1298+
release(allocated.getId());
1299+
}
12981300
}
12991301

13001302
private ContainerRequest assignWithoutLocality(Container allocated) {

hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -97,13 +97,13 @@ public abstract class RMContainerRequestor extends RMCommunicator {
9797
private final Set<ResourceRequest> requestLimitsToUpdate =
9898
new TreeSet<ResourceRequest>(RESOURCE_REQUEST_COMPARATOR);
9999

100-
private boolean nodeBlacklistingEnabled;
101-
private int blacklistDisablePercent;
100+
private volatile boolean nodeBlacklistingEnabled;
101+
private volatile int blacklistDisablePercent;
102102
private AtomicBoolean ignoreBlacklisting = new AtomicBoolean(false);
103103
private int blacklistedNodeCount = 0;
104-
private int lastClusterNmCount = 0;
105-
private int clusterNmCount = 0;
106-
private int maxTaskFailuresPerNode;
104+
private volatile int lastClusterNmCount = 0;
105+
private volatile int clusterNmCount = 0;
106+
private volatile int maxTaskFailuresPerNode;
107107
private final Map<String, Integer> nodeFailures = new HashMap<String, Integer>();
108108
private final Set<String> blacklistedNodes = Collections
109109
.newSetFromMap(new ConcurrentHashMap<String, Boolean>());

0 commit comments

Comments
 (0)