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

Plamen Jeliazkov commented on HDFS-13688:
-----------------------------------------

Hi Chen,

Couple questions for you here.

(1) I am a little confused about this part here: "fresh client who has no 
initial state id, the client needs to check AlignmentContext, to see if state 
id is set".
Why do we need to check AlignmentContext if we are a fresh client? Given that 
you have no knowledge of the state of the cluster the client should instead 
learn the stateID when it eventually decides to do something, like a read or a 
write. IMO, there should be no need for a fresh client to try to check the 
stateID. Maybe I am not understanding something.

(2) "see if state id is set, if not, DFSClient makes a RPC call to ANN".
Observer and SNN also provide stateIDs from reads. Is there a reason you need 
the stateID from the ANN?

(3) Have we considered having the txid to wait for be a parameter to msync? 
Something like {{msync(long txidToWaitFor)}}?
I can see a naive implementation being just to have the receiving server block 
until the expected txid value is reached, and then reply.
This would allow us to focus on how to supply that txid as a separate issue and 
enable folks to write their own out of bound, third party, stateID transfer 
mechanisms.

> 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