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 Fri, 22 Jun 2018 21:45:00 GMT

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

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

Post a WIP patch for early review. This patch depends on HDFS-12976, and needs to be applied
on top of HDFS-12976 v002 patch.

Some notes on the patch for reviewers, comments are welcome!:
# introduced per dfsclient AlignmentContext instance, which gets passed proxy provider. Existing
code ensures that all proxies created from this provider will have this alignment context
instance. 
# when server sets the last seen id in the rpc response, changed from lastWritten id to lastAppliedOrWrittenID
# currently, using a local spin loop with 1ms interval, at most 1000 loops to wait for observer
to catch up.
# leverage deferred response and a dedicated new thread pool of 10 thread to handle all msync,
such that handler threads will not be handling (and potentially blocking) on msync call. 10
is hard coded, can be made configurable if more preferred.
# currently, this is a call exposed through DFSClient and DistributedFilesystem, still needs
to be called explicitly. Will need to make it that every single call to Observer is somehow
piggybacked with msync.
# for a client that already has a state id set in its alignmentContext, the msync call will
directly calls into observer node to sync on this state id. But if there is no state id set
in alignmentContext (e.g. a freshly started client). The client needs to first get the current
state id from active NN, by making a "setup" call. Based on offline discussion with Konstantin,
we may not have to introduce a new "setup" call. This can be done by making any call, as long
as it is to active. Currently in ClientProtocol, there is getQuotaUsage which is annotated
with activeOnly = true. So the current patch makes a getQuotaUsage call on root directory
as a "setup" call.

> 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.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