Return-Path: X-Original-To: apmail-hadoop-yarn-issues-archive@minotaur.apache.org Delivered-To: apmail-hadoop-yarn-issues-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 5ED59928B for ; Sat, 13 Dec 2014 01:54:15 +0000 (UTC) Received: (qmail 15311 invoked by uid 500); 13 Dec 2014 01:54:13 -0000 Delivered-To: apmail-hadoop-yarn-issues-archive@hadoop.apache.org Received: (qmail 15263 invoked by uid 500); 13 Dec 2014 01:54:13 -0000 Mailing-List: contact yarn-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: yarn-issues@hadoop.apache.org Delivered-To: mailing list yarn-issues@hadoop.apache.org Received: (qmail 15252 invoked by uid 99); 13 Dec 2014 01:54:13 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 13 Dec 2014 01:54:13 +0000 Date: Sat, 13 Dec 2014 01:54:13 +0000 (UTC) From: "Varun Saxena (JIRA)" To: yarn-issues@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Assigned] (YARN-2958) RMStateStore seems to unnecessarily and wronly store sequence number separately MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/YARN-2958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Varun Saxena reassigned YARN-2958: ---------------------------------- Assignee: Varun Saxena > RMStateStore seems to unnecessarily and wronly store sequence number separately > ------------------------------------------------------------------------------- > > Key: YARN-2958 > URL: https://issues.apache.org/jira/browse/YARN-2958 > Project: Hadoop YARN > Issue Type: Bug > Components: resourcemanager > Reporter: Zhijie Shen > Assignee: Varun Saxena > > It seems that RMStateStore updates last sequence number when storing or updating each individual DT, to recover the latest sequence number when RM restarting. > First, the current logic seems to be problematic: > {code} > public synchronized void updateRMDelegationTokenAndSequenceNumber( > RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate, > int latestSequenceNumber) { > if(isFencedState()) { > LOG.info("State store is in Fenced state. Can't update RM Delegation Token."); > return; > } > try { > updateRMDelegationTokenAndSequenceNumberInternal(rmDTIdentifier, renewDate, > latestSequenceNumber); > } catch (Exception e) { > notifyStoreOperationFailed(e); > } > } > {code} > {code} > @Override > protected void updateStoredToken(RMDelegationTokenIdentifier id, > long renewDate) { > try { > LOG.info("updating RMDelegation token with sequence number: " > + id.getSequenceNumber()); > rmContext.getStateStore().updateRMDelegationTokenAndSequenceNumber(id, > renewDate, id.getSequenceNumber()); > } catch (Exception e) { > LOG.error("Error in updating persisted RMDelegationToken with sequence number: " > + id.getSequenceNumber()); > ExitUtil.terminate(1, e); > } > } > {code} > According to code above, even when renewing a DT, the last sequence number is updated in the store, which is wrong. For example, we have the following sequence: > 1. Get DT 1 (seq = 1) > 2. Get DT 2( seq = 2) > 3. Renew DT 1 (seq = 1) > 4. Restart RM > The stored and then recovered last sequence number is 1. It makes the next created DT after RM restarting will conflict with DT 2 on sequence num. > Second, the aforementioned bug doesn't happen actually, because the recovered last sequence num has been overwritten at by the correctly one. > {code} > public void recover(RMState rmState) throws Exception { > LOG.info("recovering RMDelegationTokenSecretManager."); > // recover RMDTMasterKeys > for (DelegationKey dtKey : rmState.getRMDTSecretManagerState() > .getMasterKeyState()) { > addKey(dtKey); > } > // recover RMDelegationTokens > Map rmDelegationTokens = > rmState.getRMDTSecretManagerState().getTokenState(); > this.delegationTokenSequenceNumber = > rmState.getRMDTSecretManagerState().getDTSequenceNumber(); > for (Map.Entry entry : rmDelegationTokens > .entrySet()) { > addPersistedDelegationToken(entry.getKey(), entry.getValue()); > } > } > {code} > The code above recovers delegationTokenSequenceNumber by reading the last sequence number in the store. It could be wrong. Fortunately, delegationTokenSequenceNumber updates it to the right number. > {code} > if (identifier.getSequenceNumber() > getDelegationTokenSeqNum()) { > setDelegationTokenSeqNum(identifier.getSequenceNumber()); > } > {code} > All the stored identifiers will be gone through, and delegationTokenSequenceNumber will be set to the largest sequence number among these identifiers. Therefore, new DT will be assigned a sequence number which is always larger than that of all the recovered DT. > To sum up, two negatives make a positive, but it's good to fix the issue. Please let me know if I've missed something here. -- This message was sent by Atlassian JIRA (v6.3.4#6332)