hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Chen Liang (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-13688) Introduce msync API call
Date Tue, 24 Jul 2018 22:48:00 GMT

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

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

Had an offline discussion with [~shv], updated v003 patch. Some notes/updates here:
1. decided to break the msync implementation into two parts: client side and server side.
This JIRA is specifically addressing only the client side by introducing msync API to {{ClientProtocol}}.
The server side is completely missing. This is also the reason why I did not include unit
test in v003 patch. As no meaningful test can actually be done as for now, will follow up
with tests in next part. Please let me know if tests here are still preferred.
2. msync should not take {{syncTxid}} as a parameter, because it is already being passed around
via {{AlignmentContext}}. Removing syncTxid significantly simplified client, as compared to
v002 patch.
3. In addition to having a explicit msync API call, msync should also by default be implicitly
done for every single call to Observer in order to maintain consistency. Using deferred response
can still achieve this, but it would mean using deferred response for every single read API
call. Which is not ideal. An alternative way would be leveraging callQueue so that if server
hasn't caught up with client id, the call stays in the queue. The next JIRA will be implementing
this approach.

> 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.001.patch, HDFS-13688-HDFS-12943.002.patch,
HDFS-13688-HDFS-12943.002.patch, HDFS-13688-HDFS-12943.003.patch, 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-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org


Mime
View raw message