[ https://issues.apache.org/jira/browse/HDFS-13688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16529539#comment-16529539 ]
Yiqun Lin commented on HDFS-13688: ---------------------------------- Thanks for the sharing, [~vagarychen]! {quote} 1. it is preferred not to have AlignmentContext in DFSClient. The current reason AlignmentContext in DFSClient as in WIP patch is that for a fresh client who has no initial state id, the client needs to check AlignmentContext, to see if state id is set, if not, DFSClient makes a RPC call to ANN. Since DFSClient has to make this check, it needs to see the AlignmentContext instance. If there is an alternative way where DFSClient does not need to explicitly make this check, there is no need to have AlignmentContext in DFSClient. Still need to investigate if there is an alternative way though. {quote} I am thinking for this, actually the key point now is that we should let SBN catch up with ANN. From my understanding of this, there are two approaches to achieve this. * Hold state id in client (we are assume this id is latest tx id requested from ANN), and then pass to SBN, and let SBN to reach to this desired id. * Client don't need to hold the state id and just make the msync call to SBN. But in the mysnc dealing in SBN, it will do one additional call for getting latest tx id from ANN, then wait until itself to catch up with this id. In the WIP patch, we use the first approach. Comparing with the first approach, the benefit of the second approach is that we can make client logic more simple and don't need to hold the state id. Instead, SBN will make one additional RPC call to get latest tx id from ANN every time. But if we often use the fresh client to do the msync, it should be almost same since the fresh client will also need to request the initial call to ANN. > Introduce msync API call > ------------------------ > > Key: HDFS-13688 > URL: https://issues.apache.org/jira/browse/HDFS-13688 > Project: Hadoop HDFS > Issue Type: Sub-task > Reporter: Chen Liang > Assignee: Chen Liang > Priority: Major > Attachments: HDFS-13688-HDFS-12943.WIP.002.patch, > HDFS-13688-HDFS-12943.WIP.patch > > > As mentioned in the design doc in HDFS-12943, to ensure consistent read, we > need to introduce an RPC call {{msync}}. Specifically, client can issue a > msync call to Observer node along with a transactionID. The msync will only > return when the Observer's transactionID has caught up to the given ID. This > JIRA is to add this API. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org