[jira] [Comment Edited] (HDFS-13688) Introduce msync API call

2018-06-25 Thread Yiqun Lin (JIRA)


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

Yiqun Lin edited comment on HDFS-13688 at 6/25/18 8:59 AM:
---

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. Need to 
throw the exception?

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}


was (Author: linyiqun):
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-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Comment Edited] (HDFS-13688) Introduce msync API call

2018-06-22 Thread Chen Liang (JIRA)


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

Chen Liang edited comment on HDFS-13688 at 6/22/18 9:51 PM:


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. This is based on the assumption that, with fast 
path tailing, the wait should not be very long, so the local spin might be 
sufficient and actually more efficient. This could be replaced by some other 
mechanism. 
 # 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.


was (Author: vagarychen):
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-unsubscr...@hadoop.apache.org
For additional commands, e-mail: