[ 
https://issues.apache.org/jira/browse/HDFS-13688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16522703#comment-16522703
 ] 

Chen Liang commented on HDFS-13688:
-----------------------------------

Thanks [~linyiqun] for the review, appreciate it!

bq. LastSeenId isn't tracked for both ANN and SBN.
Thanks for checking with design doc! The idea in the patch was based on some 
offline discussion we had so there seems a bit difference there. After client 
sees the ANN's state id. There were two ideas we evaluated. One is that client 
keeps sending msync calls to Observer, Observer returns immediately with its 
state id, client only returns until the observer state id catches up. A 
downside here is that multiple RPC calls are made, abusing RPC queue and 
handler CPU time on server side. 

The other approach is that client makes one single call, and Observer side will 
block the call only after the state id catches up. Since it is observer side 
making sure the id catches up, client side no longer needs to keep track of 
observer id. A downside here is that server needs more thread resources (i.e. 
the executor introduced), but I think this is a fair tradeoff compared to the 
other way.

bq. syncTnxId passed in msync call large than LastAppliedOrWrittenTxId in ANN. 
Need to throw the exception?
Fixed

bq. The condition check should be 
HAServiceState.ACTIVE.toString().equals(namesystem.getHAState()?
This led me to think of in what situation will msync be called on standby. 
Seems this happens only when there is some role transition happening, I will 
need to think of if all transition cases are properly handled here. Right now 
I'm inclined to believe the change you suggested should be sufficient. Fixed in 
the WIP.v002 patch. This is actually an interesting a point, thanks for 
bringing it up! 

bq. Why not just pass the msyncExecutor as null there?
Fixed.

> 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

Reply via email to