hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Yiqun Lin (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-13688) Introduce msync API call
Date Mon, 25 Jun 2018 08:50:00 GMT

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

Yiqun Lin commented on HDFS-13688:
----------------------------------

Hi [~vagarychen], just comparing implementation detail of msync call with that in design doc:
{noformat}
msync() implementation on the client should keep track of LastSeenId for both ANN and
SBN:
* If c.LastSeenId.ANN <= c.LastSeenId.SBN then
goto ANN and update c.LastSeenId.ANN
* Wait until SBN reaches c.LastSeenId.ANN
{noformat}
Some differences:
 * LastSeenId isn't tracked for both ANN and SBN.
 * For the corner case, the client request to ANN, meanwhile the syncTnxId passed in msync
call large than {{LastAppliedOrWrittenTxId}} in ANN. Current processing logic is different
with designed way.

Besides, for the following logic:
{code:java}
+    if (!HAServiceState.OBSERVER.toString().equals(namesystem.getHAState())) {
+      LOG.warn("Calling msync on a non-observer node:" +
+          namesystem.getHAState());
+      return namesystem.getFSImage().getLastAppliedOrWrittenTxId();
+    }
{code}
The condition check should be {{HAServiceState.ACTIVE.toString().equals(namesystem.getHAState()}}?
This is mean that only when we request for ANN, then return current txid. For the SBN/Observer
Node, we wait until catching up.

For the msync call dealing in RBF, currently we don't supported. Why not just pass the msyncExecutor
as null there? Actually it isn't real used.
{code:java}
@@ -252,9 +257,11 @@ public RouterRpcServer(Configuration configuration, Router router,
     RPC.setProtocolEngine(this.conf, ClientNamenodeProtocolPB.class,
         ProtobufRpcEngine.class);
 
+    this.msyncExecutor = Executors.newFixedThreadPool(10);
     ClientNamenodeProtocolServerSideTranslatorPB
         clientProtocolServerTranslator =
-            new ClientNamenodeProtocolServerSideTranslatorPB(this);
+            new ClientNamenodeProtocolServerSideTranslatorPB(
+                this, msyncExecutor);
{code}

> 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