YARN-3480. Remove attempts that are beyond max-attempt limit from state store. Contributed by Jun Gong

(cherry picked from commit 5273413411)
This commit is contained in:
Jian He 2015-12-29 15:58:39 -08:00
parent 3efbde5378
commit 3661d4907a
18 changed files with 341 additions and 7 deletions

View File

@ -21,6 +21,9 @@ Release 2.9.0 - UNRELEASED
YARN-4417. Make RM and Timeline-server REST APIs more consistent.
(wtan via jianhe)
YARN-3480. Remove attempts that are beyond max-attempt limit from state
store. (Jun Gong via jianhe)
OPTIMIZATIONS
BUG FIXES

View File

@ -817,6 +817,33 @@ public class ResourceManager extends CompositeService implements Recoverable {
LOG.error("Error in handling event type " + event.getType()
+ " for applicationAttempt " + appAttemptId, t);
}
} else if (rmApp.getApplicationSubmissionContext() != null
&& rmApp.getApplicationSubmissionContext()
.getKeepContainersAcrossApplicationAttempts()
&& event.getType() == RMAppAttemptEventType.CONTAINER_FINISHED) {
// For work-preserving AM restart, failed attempts are still
// capturing CONTAINER_FINISHED events and record the finished
// containers which will be used by current attempt.
// We just keep 'yarn.resourcemanager.am.max-attempts' in
// RMStateStore. If the finished container's attempt is deleted, we
// use the first attempt in app.attempts to deal with these events.
RMAppAttempt previousFailedAttempt =
rmApp.getAppAttempts().values().iterator().next();
if (previousFailedAttempt != null) {
try {
LOG.debug("Event " + event.getType() + " handled by "
+ previousFailedAttempt);
previousFailedAttempt.handle(event);
} catch (Throwable t) {
LOG.error("Error in handling event type " + event.getType()
+ " for applicationAttempt " + appAttemptId
+ " with " + previousFailedAttempt, t);
}
} else {
LOG.error("Event " + event.getType()
+ " not handled, because previousFailedAttempt is null");
}
}
}
}

View File

@ -230,6 +230,11 @@ public class ResourceTrackerService extends AbstractService implements
}
RMAppAttempt rmAppAttempt = rmApp.getRMAppAttempt(appAttemptId);
if (rmAppAttempt == null) {
LOG.info("Ignoring not found attempt " + appAttemptId);
return;
}
Container masterContainer = rmAppAttempt.getMasterContainer();
if (masterContainer.getId().equals(containerStatus.getContainerId())
&& containerStatus.getContainerState() == ContainerState.COMPLETE) {

View File

@ -481,6 +481,18 @@ public class FileSystemRMStateStore extends RMStateStore {
}
}
@Override
public synchronized void removeApplicationAttemptInternal(
ApplicationAttemptId appAttemptId)
throws Exception {
Path appDirPath =
getAppDir(rmAppRoot, appAttemptId.getApplicationId());
Path nodeRemovePath = getNodePath(appDirPath, appAttemptId.toString());
LOG.info("Removing info for attempt: " + appAttemptId + " at: "
+ nodeRemovePath);
deleteFileWithRetries(nodeRemovePath);
}
@Override
public synchronized void removeApplicationStateInternal(
ApplicationStateData appState)

View File

@ -500,6 +500,22 @@ public class LeveldbRMStateStore extends RMStateStore {
return createApplicationState(appId.toString(), data);
}
@VisibleForTesting
ApplicationAttemptStateData loadRMAppAttemptState(
ApplicationAttemptId attemptId) throws IOException {
String attemptKey = getApplicationAttemptNodeKey(attemptId);
byte[] data = null;
try {
data = db.get(bytes(attemptKey));
} catch (DBException e) {
throw new IOException(e);
}
if (data == null) {
return null;
}
return createAttemptState(attemptId.toString(), data);
}
private ApplicationAttemptStateData createAttemptState(String itemName,
byte[] data) throws IOException {
ApplicationAttemptId attemptId =
@ -574,6 +590,22 @@ public class LeveldbRMStateStore extends RMStateStore {
storeApplicationAttemptStateInternal(attemptId, attemptStateData);
}
@Override
public synchronized void removeApplicationAttemptInternal(
ApplicationAttemptId attemptId)
throws IOException {
String attemptKey = getApplicationAttemptNodeKey(attemptId);
if (LOG.isDebugEnabled()) {
LOG.debug("Removing state for attempt " + attemptId + " at "
+ attemptKey);
}
try {
db.delete(bytes(attemptKey));
} catch (DBException e) {
throw new IOException(e);
}
}
@Override
protected void removeApplicationStateInternal(ApplicationStateData appState)
throws IOException {

View File

@ -141,6 +141,19 @@ public class MemoryRMStateStore extends RMStateStore {
appState.attempts.put(attemptState.getAttemptId(), attemptState);
}
@Override
public synchronized void removeApplicationAttemptInternal(
ApplicationAttemptId appAttemptId) throws Exception {
ApplicationStateData appState =
state.getApplicationState().get(appAttemptId.getApplicationId());
ApplicationAttemptStateData attemptState =
appState.attempts.remove(appAttemptId);
LOG.info("Removing state for attempt: " + appAttemptId);
if (attemptState == null) {
throw new YarnRuntimeException("Application doesn't exist");
}
}
@Override
public synchronized void removeApplicationStateInternal(
ApplicationStateData appState) throws Exception {

View File

@ -131,6 +131,12 @@ public class NullRMStateStore extends RMStateStore {
ApplicationAttemptStateData attemptStateData) throws Exception {
}
@Override
public synchronized void removeApplicationAttemptInternal(
ApplicationAttemptId attemptId) throws Exception {
// Do nothing
}
@Override
public void checkVersion() throws Exception {
// Do nothing

View File

@ -135,6 +135,10 @@ public abstract class RMStateStore extends AbstractService {
EnumSet.of(RMStateStoreState.ACTIVE, RMStateStoreState.FENCED),
RMStateStoreEventType.UPDATE_APP_ATTEMPT,
new UpdateAppAttemptTransition())
.addTransition(RMStateStoreState.ACTIVE,
EnumSet.of(RMStateStoreState.ACTIVE, RMStateStoreState.FENCED),
RMStateStoreEventType.REMOVE_APP_ATTEMPT,
new RemoveAppAttemptTransition())
.addTransition(RMStateStoreState.ACTIVE,
EnumSet.of(RMStateStoreState.ACTIVE, RMStateStoreState.FENCED),
RMStateStoreEventType.STORE_MASTERKEY,
@ -552,6 +556,32 @@ public abstract class RMStateStore extends AbstractService {
return isFenced ? RMStateStoreState.FENCED : RMStateStoreState.ACTIVE;
}
private static class RemoveAppAttemptTransition implements
MultipleArcTransition<RMStateStore, RMStateStoreEvent,
RMStateStoreState> {
@Override
public RMStateStoreState transition(RMStateStore store,
RMStateStoreEvent event) {
if (!(event instanceof RMStateStoreRemoveAppAttemptEvent)) {
// should never happen
LOG.error("Illegal event type: " + event.getClass());
return RMStateStoreState.ACTIVE;
}
boolean isFenced = false;
ApplicationAttemptId attemptId =
((RMStateStoreRemoveAppAttemptEvent) event).getApplicationAttemptId();
ApplicationId appId = attemptId.getApplicationId();
LOG.info("Removing attempt " + attemptId + " from app: " + appId);
try {
store.removeApplicationAttemptInternal(attemptId);
} catch (Exception e) {
LOG.error("Error removing attempt: " + attemptId, e);
isFenced = store.notifyStoreOperationFailedInternal(e);
}
return finalState(isFenced);
}
}
public RMStateStore() {
super(RMStateStore.class.getName());
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
@ -983,6 +1013,29 @@ public abstract class RMStateStore extends AbstractService {
protected abstract void removeApplicationStateInternal(
ApplicationStateData appState) throws Exception;
/**
* Non-blocking API
* ResourceManager services call this to remove an attempt from the state
* store
* This does not block the dispatcher threads
* There is no notification of completion for this operation.
*/
@SuppressWarnings("unchecked")
public synchronized void removeApplicationAttempt(
ApplicationAttemptId applicationAttemptId) {
dispatcher.getEventHandler().handle(
new RMStateStoreRemoveAppAttemptEvent(applicationAttemptId));
}
/**
* Blocking API
* Derived classes must implement this method to remove the state of specified
* attempt.
*/
protected abstract void removeApplicationAttemptInternal(
ApplicationAttemptId attemptId) throws Exception;
// TODO: This should eventually become cluster-Id + "AM_RM_TOKEN_SERVICE". See
// YARN-1779
public static final Text AM_RM_TOKEN_SERVICE = new Text(

View File

@ -24,6 +24,7 @@ public enum RMStateStoreEventType {
UPDATE_APP,
UPDATE_APP_ATTEMPT,
REMOVE_APP,
REMOVE_APP_ATTEMPT,
FENCED,
// Below events should be called synchronously

View File

@ -0,0 +1,37 @@
/**
* 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.yarn.server.resourcemanager.recovery;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
/**
* A event used to remove an attempt.
*/
public class RMStateStoreRemoveAppAttemptEvent extends RMStateStoreEvent {
private ApplicationAttemptId applicationAttemptId;
RMStateStoreRemoveAppAttemptEvent(ApplicationAttemptId applicationAttemptId) {
super(RMStateStoreEventType.REMOVE_APP_ATTEMPT);
this.applicationAttemptId = applicationAttemptId;
}
public ApplicationAttemptId getApplicationAttemptId() {
return applicationAttemptId;
}
}

View File

@ -658,6 +658,22 @@ public class ZKRMStateStore extends RMStateStore {
}
}
@Override
public synchronized void removeApplicationAttemptInternal(
ApplicationAttemptId appAttemptId)
throws Exception {
String appId = appAttemptId.getApplicationId().toString();
String appIdRemovePath = getNodePath(rmAppRoot, appId);
String attemptIdRemovePath = getNodePath(appIdRemovePath,
appAttemptId.toString());
if (LOG.isDebugEnabled()) {
LOG.debug("Removing info for attempt: " + appAttemptId + " at: "
+ attemptIdRemovePath);
}
safeDelete(attemptIdRemovePath);
}
@Override
public synchronized void removeApplicationStateInternal(
ApplicationStateData appState)

View File

@ -80,6 +80,16 @@ public abstract class ApplicationStateData {
return attempts.get(attemptId);
}
public int getFirstAttemptId() {
int min = Integer.MAX_VALUE;
for(ApplicationAttemptId attemptId : attempts.keySet()) {
if (attemptId.getAttemptId() < min) {
min = attemptId.getAttemptId();
}
}
return min == Integer.MAX_VALUE ? 1 : min;
}
public abstract ApplicationStateDataProto getProto();
/**

View File

@ -149,6 +149,8 @@ public class RMAppImpl implements RMApp, Recoverable {
private long startTime;
private long finishTime = 0;
private long storedFinishTime = 0;
private int firstAttemptIdInStateStore = 1;
private int nextAttemptId = 1;
// This field isn't protected by readlock now.
private volatile RMAppAttempt currentAttempt;
private String queue;
@ -809,6 +811,11 @@ public class RMAppImpl implements RMApp, Recoverable {
this.storedFinishTime = appState.getFinishTime();
this.startTime = appState.getStartTime();
this.callerContext = appState.getCallerContext();
// If interval > 0, some attempts might have been deleted.
if (submissionContext.getAttemptFailuresValidityInterval() > 0) {
this.firstAttemptIdInStateStore = appState.getFirstAttemptId();
this.nextAttemptId = firstAttemptIdInStateStore;
}
// send the ATS create Event
sendATSCreateEvent(this, this.startTime);
@ -822,7 +829,7 @@ public class RMAppImpl implements RMApp, Recoverable {
private void createNewAttempt() {
ApplicationAttemptId appAttemptId =
ApplicationAttemptId.newInstance(applicationId, attempts.size() + 1);
ApplicationAttemptId.newInstance(applicationId, nextAttemptId++);
BlacklistManager currentAMBlacklist;
if (currentAttempt != null) {
@ -1304,6 +1311,9 @@ public class RMAppImpl implements RMApp, Recoverable {
+ app.attemptFailuresValidityInterval + " milliseconds " : " ")
+ "is " + numberOfFailure + ". The max attempts is "
+ app.maxAppAttempts);
removeExcessAttempts(app);
if (!app.submissionContext.getUnmanagedAM()
&& numberOfFailure < app.maxAppAttempts) {
if (initialState.equals(RMAppState.KILLING)) {
@ -1340,6 +1350,19 @@ public class RMAppImpl implements RMApp, Recoverable {
return RMAppState.FINAL_SAVING;
}
}
private void removeExcessAttempts(RMAppImpl app) {
while (app.nextAttemptId - app.firstAttemptIdInStateStore
> app.maxAppAttempts) {
// attempts' first element is oldest attempt because it is a
// LinkedHashMap
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
app.getApplicationId(), app.firstAttemptIdInStateStore);
app.firstAttemptIdInStateStore++;
LOG.info("Remove attempt from state store : " + attemptId);
app.rmContext.getStateStore().removeApplicationAttempt(attemptId);
}
}
}
@Override

View File

@ -859,6 +859,9 @@ public class TestAMRestart {
@SuppressWarnings("resource")
MockRM rm2 = new MockRM(conf, memStore);
rm2.start();
ApplicationStateData app1State =
memStore.getState().getApplicationState().get(app1.getApplicationId());
Assert.assertEquals(1, app1State.getFirstAttemptId());
// re-register the NM
nm1.setResourceTrackerService(rm2.getResourceTrackerService());
@ -871,6 +874,7 @@ public class TestAMRestart {
nm1.registerNode(Collections.singletonList(status), null);
rm2.waitForState(attempt3.getAppAttemptId(), RMAppAttemptState.FAILED);
Assert.assertEquals(2, app1State.getAttemptCount());
rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
@ -884,6 +888,7 @@ public class TestAMRestart {
nm1
.nodeHeartbeat(am4.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am4.waitForState(RMAppAttemptState.FAILED);
Assert.assertEquals(2, app1State.getAttemptCount());
// can launch the 5th attempt successfully
rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
@ -897,6 +902,7 @@ public class TestAMRestart {
nm1
.nodeHeartbeat(am5.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
am5.waitForState(RMAppAttemptState.FAILED);
Assert.assertEquals(2, app1State.getAttemptCount());
rm2.waitForState(app1.getApplicationId(), RMAppState.FAILED);
rm1.stop();

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.recovery;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
@ -134,6 +135,7 @@ public class RMStateStoreTestBase {
void writeVersion(Version version) throws Exception;
Version getCurrentVersion() throws Exception;
boolean appExists(RMApp app) throws Exception;
boolean attemptExists(RMAppAttempt attempt) throws Exception;
}
void waitNotify(TestDispatcher dispatcher) {
@ -172,7 +174,7 @@ public class RMStateStoreTestBase {
return mockApp;
}
protected ContainerId storeAttempt(RMStateStore store,
protected RMAppAttempt storeAttempt(RMStateStore store,
ApplicationAttemptId attemptId,
String containerIdStr, Token<AMRMTokenIdentifier> appToken,
SecretKey clientTokenMasterKey, TestDispatcher dispatcher)
@ -195,7 +197,7 @@ public class RMStateStoreTestBase {
dispatcher.attemptId = attemptId;
store.storeNewApplicationAttempt(mockAttempt);
waitNotify(dispatcher);
return container.getId();
return mockAttempt;
}
void testRMAppStateStore(RMStateStoreHelper stateStoreHelper)
@ -238,8 +240,9 @@ public class RMStateStoreTestBase {
clientToAMTokenMgr.createMasterKey(attemptId1);
ContainerId containerId1 = storeAttempt(store, attemptId1,
"container_1352994193343_0001_01_000001",
appAttemptToken1, clientTokenKey1, dispatcher);
"container_1352994193343_0001_01_000001",
appAttemptToken1, clientTokenKey1, dispatcher)
.getMasterContainer().getId();
String appAttemptIdStr2 = "appattempt_1352994193343_0001_000002";
ApplicationAttemptId attemptId2 =
@ -252,8 +255,9 @@ public class RMStateStoreTestBase {
clientToAMTokenMgr.createMasterKey(attemptId2);
ContainerId containerId2 = storeAttempt(store, attemptId2,
"container_1352994193343_0001_02_000001",
appAttemptToken2, clientTokenKey2, dispatcher);
"container_1352994193343_0001_02_000001",
appAttemptToken2, clientTokenKey2, dispatcher)
.getMasterContainer().getId();
ApplicationAttemptId attemptIdRemoved = ConverterUtils
.toApplicationAttemptId("appattempt_1352994193343_0002_000001");
@ -633,6 +637,47 @@ public class RMStateStoreTestBase {
Assert.assertTrue(stateStoreHelper.appExists(rmApp2));
}
public void testRemoveAttempt(RMStateStoreHelper stateStoreHelper)
throws Exception {
RMStateStore store = stateStoreHelper.getRMStateStore();
TestDispatcher dispatcher = new TestDispatcher();
store.setRMDispatcher(dispatcher);
ApplicationId appId = ApplicationId.newInstance(1383183339, 6);
storeApp(store, appId, 123456, 564321);
ApplicationAttemptId attemptId1 =
ApplicationAttemptId.newInstance(appId, 1);
RMAppAttempt attempt1 = storeAttempt(store, attemptId1,
ContainerId.newContainerId(attemptId1, 1).toString(),
null, null, dispatcher);
ApplicationAttemptId attemptId2 =
ApplicationAttemptId.newInstance(appId, 2);
RMAppAttempt attempt2 = storeAttempt(store, attemptId2,
ContainerId.newContainerId(attemptId2, 1).toString(),
null, null, dispatcher);
store.removeApplicationAttemptInternal(attemptId1);
Assert.assertFalse(stateStoreHelper.attemptExists(attempt1));
Assert.assertTrue(stateStoreHelper.attemptExists(attempt2));
// let things settle down
Thread.sleep(1000);
store.close();
// load state
store = stateStoreHelper.getRMStateStore();
RMState state = store.loadState();
Map<ApplicationId, ApplicationStateData> rmAppState =
state.getApplicationState();
ApplicationStateData appState = rmAppState.get(appId);
// app is loaded
assertNotNull(appState);
assertEquals(2, appState.getFirstAttemptId());
assertNull(appState.getAttempt(attemptId1));
assertNotNull(appState.getAttempt(attemptId2));
}
protected void modifyAppState() throws Exception {
}

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.junit.Assert;
import org.junit.Test;
@ -88,6 +89,12 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
Path appDir = new Path(appRootDir, appId);
return appDir;
}
public Path getAttemptDir(String appId, String attemptId) {
Path appDir = getAppDir(appId);
Path attemptDir = new Path(appDir, attemptId);
return attemptDir;
}
}
public TestFSRMStateStoreTester(MiniDFSCluster cluster, boolean adminCheckEnable) throws Exception {
@ -151,6 +158,15 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
store.getAppDir(app.getApplicationId().toString());
return fs.exists(nodePath);
}
public boolean attemptExists(RMAppAttempt attempt) throws IOException {
FileSystem fs = cluster.getFileSystem();
ApplicationAttemptId attemptId = attempt.getAppAttemptId();
Path nodePath =
store.getAttemptDir(attemptId.getApplicationId().toString(),
attemptId.toString());
return fs.exists(nodePath);
}
}
@Test(timeout = 60000)
@ -185,6 +201,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
testAppDeletion(fsTester);
testDeleteStore(fsTester);
testRemoveApplication(fsTester);
testRemoveAttempt(fsTester);
testAMRMTokenSecretManagerStateStore(fsTester);
testReservationStateStore(fsTester);
} finally {

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.records.Version;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -96,6 +97,12 @@ public class TestLeveldbRMStateStore extends RMStateStoreTestBase {
testRemoveApplication(tester);
}
@Test(timeout = 60000)
public void testRemoveAttempt() throws Exception {
LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
testRemoveAttempt(tester);
}
@Test(timeout = 60000)
public void testAMTokens() throws Exception {
LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
@ -147,5 +154,14 @@ public class TestLeveldbRMStateStore extends RMStateStoreTestBase {
}
return stateStore.loadRMAppState(app.getApplicationId()) != null;
}
@Override
public boolean attemptExists(RMAppAttempt attempt) throws Exception {
if (stateStore.isClosed()) {
getRMStateStore();
}
return stateStore.loadRMAppAttemptState(attempt.getAppAttemptId())
!= null;
}
}
}

View File

@ -123,6 +123,10 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
+ appId;
}
public String getAttemptNode(String appId, String attemptId) {
return getAppNode(appId) + "/" + attemptId;
}
/**
* Emulating retrying createRootDir not to raise NodeExist exception
* @throws Exception
@ -165,6 +169,13 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
return null != curatorFramework.checkExists()
.forPath(store.getAppNode(app.getApplicationId().toString()));
}
public boolean attemptExists(RMAppAttempt attempt) throws Exception {
ApplicationAttemptId attemptId = attempt.getAppAttemptId();
return null != curatorFramework.checkExists()
.forPath(store.getAttemptNode(
attemptId.getApplicationId().toString(), attemptId.toString()));
}
}
@Test (timeout = 60000)
@ -177,6 +188,7 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
testAppDeletion(zkTester);
testDeleteStore(zkTester);
testRemoveApplication(zkTester);
testRemoveAttempt(zkTester);
testAMRMTokenSecretManagerStateStore(zkTester);
testReservationStateStore(zkTester);
((TestZKRMStateStoreTester.TestZKRMStateStoreInternal)