[ https://issues.apache.org/jira/browse/HDFS-13688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16545898#comment-16545898 ]
Chen Liang commented on HDFS-13688: ----------------------------------- I was rebasing the patch, and I found that one of my tests failed. After looking into it, it seems that it was caused by that in the final committed version of HDFS-12976, {{isObserverState}} has changed. It gets called on initialization of {{ObserverReadProxyProvider}}. In current new {{isObserverState}} call, it makes an RPC call to every NameNode. As a side effect, this RPC call will also set up client side state id. In which case, even for a fresh client, it will have state id set after {{ObserverReadProxyProvider}} gets created, so there is no longer the need to explicitly making a call to set state id for fresh clients. I will post another patch to reflect this change. > 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