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

Ivan Andika commented on HDDS-13933:
------------------------------------

I recently found that even the HDFS consistent read is not good enough for our 
use case and user need to implement some workaround using CallerContext to 
trigger msync to ensure that they are reading the newest data.

Therefore, I will icebox this first. We can focus on the linearizable read 
(strong consistency / read-after-write consistency) and bounded stale reads.

> Consistent Read from OM Followers
> ---------------------------------
>
>                 Key: HDDS-13933
>                 URL: https://issues.apache.org/jira/browse/HDDS-13933
>             Project: Apache Ozone
>          Issue Type: New Feature
>            Reporter: Ivan Andika
>            Assignee: Ivan Andika
>            Priority: Major
>
> HDDS-9279 introduces the Ratis / Raft based follower reads by giving an 
> option to enable Ratis Linearizable Read and Leader Lease feature. However, 
> based on previous performance tests, there are significant performance 
> regressions both in the client and server which makes it not production 
> ready. Currently, the root cause of this performance hit has yet to be 
> discovered.
> Therefore, I suggest we pursue two concurrent approaches for consistent 
> follower read. We can then compare the two approach and enable the one with 
> the better performance.
>  * Improve the Ratis Lineariable Read and ReadIndex improvements (following 
> up on HDDS-9279)
>  ** In my opinion, since we are stuck on improving this, we might try the 
> second approach
>  * (This ticket) We follow the HDFS observer read implementation (HDFS-12943)
>  ** General flow
>  *** Client send an msync to OM leader and OM leader reply with the last 
> applied index as part of the AlignmentContext#getLastSeenId
>  *** When client send a request to the OM follower, the Hadoop RPC mechanism 
> will detect the AlignmentContext and will requeue the call if the 
> Call#getClientStateId() > AlignmentContext.getLastSeenStateId() (see Hadoop 
> Server.Handler#run)
>  **** This implies that the server RPC handler will not get blocked, unlike 
> linearizable read
>  ** Pros
>  *** There is already a reference implementation on HDFS, we can simply 
> finesse it to Ozone context
>  **** e.g. FSImage#getLastAppliedOrWrittenTxId can be translated to 
> StateMachine#getLastAppliedTermIndex
>  *** This is similar to HDFS observer read implementation, so we know that 
> this level of consistency is acceptable and we don't need a lot to prove that 
> it is correct (or at least acceptable)
>  **** If we know that HDFS observer read has been deployed to production with 
> no consistency issue and with acceptable performance so we expect the same on 
> Ozone
>  *** Possible better performance since Server will requeue the call to the 
> RPC server instead of blocking
>  *** We can adapt the Client proxy provider implementation from 
> ObserverProxyProvider
>  ** Cons
>  *** Only supports Hadoop RPC based client (gRPC based client is not 
> supported and requires its own development) 
>  *** Will drift the implementation away from Raft / Ratis
>  ** Current Implementation plans
>  *** Ozone side 
>  **** AlignmentContext for client
>  ***** Implementation is similar to Hadoop ClientGSIContext
>  **** AlignmentContext for OM
>  ***** Implementation is similar to Hadoop GlobalStateIdContext
>  **** Create a new proxy provider similar to ObserverReadProxyProvider to 
> allow to route read requests to the OM followers
>  *** Hadoop side
>  **** Support parsing AlignmentContext for ProtobufRpcEngine (raised 
> HADOOP-19741)
>  ***** See Hadoop Server.Connection#processRequest
> Additionally I hope that implementing one can uncover issues and improvements 
> on the other.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to