You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pegasus.apache.org by GitBox <gi...@apache.org> on 2021/11/18 02:20:59 UTC

[GitHub] [incubator-pegasus] cauchy1988 opened a new issue #840: add 'feature flags' rcp interface for version compatability between server and client

cauchy1988 opened a new issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840


   ## Feature Request
   
   **Is your feature request related to a problem? Please describe:**
   <!-- A clear and concise description of what the problem is. Ex. I'm always frustrated when [...] -->
   
   We have optimized an interface implementation of the pegasus java  client. The specific way is to define a new 'task_code' 、bind it to a new read interface function implemented by the server side and then reimplement the interface of the java client by calling the new read rpc interface represented by the new 'task_code'; but this will cause the problem of version matching between the client side and the server side: the new version client can not visit old version server because old version server cannot recognize the new  task-code in the request-struct;
    
   Since my company is a "to b" company, we can not upgrade all the deployed pegasus-server cluster in a comparatively short time;
    We don't want to influence the usage of the old client interface either,  so we implement a new java client interface instead with the same functionality and make it compatible with the new and old version server at the same time.
    
   How to implement the client interface in compatible with the new and old version server at the same time ? 
    
   We have once used an unconsidered method:
   we added a 'version'  field  which represents server code version to  the 'configuration_query_by_index_response'  structure returned  from the metaserver in the rpc-call represented by the task-code 'RPC_CM_QUERY_PARTITION_CONFIG_BY_INDEX' and then the java client interface implementation can do the corresponding  logic based by this fetched 'version' field: if ‘version' is higher than some pre-specified number, the remain logic will execute new optimized code and call the new rpc read interface ; otherwise, it will be the same with the old logic
    
   This  'unconsidered' method is called 'unconsidered' because it  has the following problems:
   (1) 'version' field can only represents metaserver's version, not all the servers' version in the same pegasus cluster, especially when upgrading is in progress
   (2) it is unreasonble to use an  unrelated rpc-call and semantically unrelated thrift structure 'configuration_query_by_index_response' to fetch 'version' field
   (3) It is very inflexible to judge the features supported by the server simply by comparing  the 'code version'
    
   **Describe the feature you'd like:**
   <!-- A clear and concise description of what you want to happen. -->
   
   So later we realized that the version matching problem is a general problem, we thought of a more general method which we call it 'feature flags method', and the idea is inspired from 'kudu', the method is as following:
   1、we can define new features as enums coded in the client side and server side at the same time
   enum rpc_feature_flag {
   UNKNOWN = 0,  
           FEATURE_1 = 1,
   }
   2、we can also define a new 'task_code', for example 'RPC_CM_FEATURE_NEGOTIATE' and corresponding rpc related definitions:
   struct negotiate_request {
    }
   struct negotiate_response {  
       1:list<rpc_feature_flag>  supported_features;
    }
   service rrdb {
       negotiate_response negotiate(1:negotiate_request request);
   }
   3、we can then define a new class inherited from 'serverlet' , implement the real rpc handler function and register it to 'rpc_engine'  in this class when related 'service_app' start
    
   4、when a java or other language's  client  establishing connection with each instance of the server side, it also do the 'negotiate' rpc call to each of them, then it can 'know'  the features each server supported, thus can make next decisions
    
    
   The above 'feature flags  method'  is not particularly detailed, for example it have not  described what the client would do when one of a  'negotiate' rpc  fails
   I put it forward here, because I want to hear the opinions of the great gods here!
   
   
   **Describe alternatives you've considered:**
   <!-- A clear and concise description of any alternative solutions or features you've considered. -->
   
   **Teachability, Documentation, Adoption, Migration Strategy:**
   <!-- If you can, explain some scenarios how users might use this, situations it would be helpful in. Any API designs, mockups, or diagrams are also helpful. -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Shuo-Jia commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Shuo-Jia commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018399256


   > meta server should remember the feature list of all the replica server
   
   Well, I see, you can select to resolve it based different cases. Another resolution is  the response just fill the `alive node`when query `feature_list`:
   ```c++
   node_state _state; // global, `node_state` is regarded as latest `nodes-map config` in rdsn
   
   feature_map _feature; // global, it may include expired node or config;
   
   feature_map resp;// the actually need resp
   for(n: node_state) {
   resp.put(n, _feature[n]);
   }
   ```
   
   client can triger update if the config is still older and receive error. It's also say, the config update is lazy, which no need be precisely
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1006316481


   > If you still want to handle the replica version, you can let meta to sync the replica server in `config_sync` and return the client.
   > 
   > In short, I think it will be clearer if you only deal with the interaction with meta
   
   it's really a simpler way; let me think for a while


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-986480900


   > It's a good design, but why not use `version` to determine which implementation to use
   
   This is also a solution, but you need to remember the mapping between each version and it's corresponding featurelist in both client and server side


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 edited a comment on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 edited a comment on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1006475468


   > If you still want to handle the replica version, you can let meta to sync the replica server in `config_sync` and return the client.
   > 
   > In short, I think it will be clearer if you only deal with the interaction with meta
   
   implement this issue in this way ? @acelyc111 @hycdong @Smityz @Shuo-Jia  Any of you have any comments?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 edited a comment on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 edited a comment on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018384900


   > ## 1
   > > The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically...
   > 
   > Refer to implementation `config update`, you just update after the response of replica is `no support`. rather than boot one scheduled task
   > 
   > ## 2
   > > The server implementation is more complicated and needs to be well tested
   > 
   > You still need implement same or more code on server side when you use the old design which client negotiation replica server, and the new design just apply the logic to meta and you can re-use `config-sync`, so I think it shouldn't be `more complicated`.
   > 
   > You list the update case in
   > 
   > > according to three possible situations
   > 
   > Actually, you may not consider these different case, the `config-sync` just `run` and store the latest value.
   > 
   > Just like @acelyc111 say:
   > 
   > > We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'
   > 
   > ## 3
   > > so the client can only start a periodically scheduled task
   > 
   > As above, refer to implementation of `meta config` , you don't need periodically scheduled task.
   > 
   > ## 4
   > > but the business code of BATCH_GET rpc is in the pegaus
   > 
   > They are two feature, and locate different git-rep is acceptable, it will be improved after they are combined
   **1**
   I think it is still inevitable to add a scheduled timer update-process, think about this scenerio: java client fetched an already outdated feature list from leader metaserver, and this feature list doesn't contains 'BATCH_GET' feature, then java client will run in old logic; however, the whole server cluster has been upgraded to new version which support the 'BATCH_GET' feature; the dilemma is that : java client will not get the no support response from the replica because replica also support old-logic-interface, and thus the client will go on running in old logic forever........
   
   **2**
   meta server should remember the feature list of all the replica server in a 'map like' datastructure, and judge one feature is supported only when all featurelist of every replica server contains this feature;    so we should dynamically update this "map-like' datastructure  in three possible conditions i decribed 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Shuo-Jia commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Shuo-Jia commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-998712207


   If you still want to handle the replica version, you can let meta to sync the replica server in `config_sync` and return the client.
   
   In short, I think it will be clearer if you only deal with the interaction with meta


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 edited a comment on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 edited a comment on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018384900


   > ## 1
   > > The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically...
   > 
   > Refer to implementation `config update`, you just update after the response of replica is `no support`. rather than boot one scheduled task
   > 
   > ## 2
   > > The server implementation is more complicated and needs to be well tested
   > 
   > You still need implement same or more code on server side when you use the old design which client negotiation replica server, and the new design just apply the logic to meta and you can re-use `config-sync`, so I think it shouldn't be `more complicated`.
   > 
   > You list the update case in
   > 
   > > according to three possible situations
   > 
   > Actually, you may not consider these different case, the `config-sync` just `run` and store the latest value.
   > 
   > Just like @acelyc111 say:
   > 
   > > We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'
   > 
   > ## 3
   > > so the client can only start a periodically scheduled task
   > 
   > As above, refer to implementation of `meta config` , you don't need periodically scheduled task.
   > 
   > ## 4
   > > but the business code of BATCH_GET rpc is in the pegaus
   > 
   > They are two feature, and locate different git-rep is acceptable, it will be improved after they are combined
   
   
   
   **1**
   I think it is still inevitable to add a scheduled timer update-process, think about this scenerio: java client fetched an already outdated feature list from leader metaserver, and this feature list doesn't contains 'BATCH_GET' feature, then java client will run in old logic; however, the whole server cluster has been upgraded to new version which support the 'BATCH_GET' feature; the dilemma is that : java client will not get the no support response from the replica because replica also support old-logic-interface, and thus the client will go on running in old logic forever........
   
   **2**
   meta server should remember the feature list of all the replica server in a 'map like' datastructure, and judge one feature is supported only when all featurelist of every replica server contains this feature;    so we should dynamically update this "map-like' datastructure  in three possible conditions i decribed 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Shuo-Jia commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Shuo-Jia commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018401855


   > java client will not get the no support response
   
   Yeah, I see, you are right


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Smityz commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1006340639


   1. Periodic update server version in the client 
   2. The client automatically matches the corresponding implementation of the server. (like if server_version > 2.4 then use batchget3 else use batchget2)
   
   In this way, we don't need to change the codes in the server. Clients can handle everything about Negotiation.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1010681932


   > If you still want to handle the replica version, you can let meta to sync the replica server in `config_sync` and return the client.
   > 
   > In short, I think it will be clearer if you only deal with the interaction with meta
   
   I thought about how to implement it, and draw a simple picture as below
   ![image](https://user-images.githubusercontent.com/7292411/149073904-23d00661-59b3-477a-92e0-b5a1390d8964.png)
   
   The main process of implementation is divided into two parts:
   1、Interaction between leader meta server and replica server
   2、interaction between java client and leader meta server
   The two parts will be described as follows
    
   1、Interaction between leader meta server and replica server
   Leader meta server stores the list of feature lists supported by each replica server in its own local memory; at the same time, this list should be modified according to three possible situations: (1) The feature list for some replica server stored in leader meta server is different  to  feature list received from the config sync request sent by the replica server  (2) A replica server was originally not in the local memory records (it was considered to be down before) but is judged alive by the leader meta server, and its corresponding feature list record needs to be added later by corresponding config sync rpc (3) A replica server has been delayed Sending a heartbeat to the leader meta server; so it was judged dead by the leader meta server, and its entire feature list record needs to be deleted from the local memory records
    
   Situation(2) and(3) will be triggered by pegasus already realized 'failure_detector' mechanism
   and thus easy to reuse
    
   2、The interaction between java client and leader meta server
   The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically to solve this problem
   
   However This implementation still has the following problems:
   (1)The server implementation is more complicated and needs to be well tested
   (2)Since the client only pulls the  features  list from the lead meta server, the client may pull the list of old version, so the client can only start a periodically scheduled task and continuously pull the feature list from the leader meta server to update its local one
   (3)The codes which meta server interact with replica server and supported feature definition are all in the rdsn, but  the business code of BATCH_GET rpc is in the pegaus.   although in the future the two code repository will combine into one  
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018390994


   > > ## 1
   > > > The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically...
   > > 
   > > 
   > > Refer to implementation `config update`, you just update after the response of replica is `no support`. rather than boot one scheduled task
   > > ## 2
   > > > The server implementation is more complicated and needs to be well tested
   > > 
   > > 
   > > You still need implement same or more code on server side when you use the old design which client negotiation replica server, and the new design just apply the logic to meta and you can re-use `config-sync`, so I think it shouldn't be `more complicated`.
   > > You list the update case in
   > > > according to three possible situations
   > > 
   > > 
   > > Actually, you may not consider these different case, the `config-sync` just `run` and store the latest value.
   > > Just like @acelyc111 say:
   > > > We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'
   > > 
   > > 
   > > ## 3
   > > > so the client can only start a periodically scheduled task
   > > 
   > > 
   > > As above, refer to implementation of `meta config` , you don't need periodically scheduled task.
   > > ## 4
   > > > but the business code of BATCH_GET rpc is in the pegaus
   > > 
   > > 
   > > They are two feature, and locate different git-rep is acceptable, it will be improved after they are combined
   > 
   > **2** meta server should remember the feature list of all the replica server in a 'map like' datastructure, and judge one feature is supported only when all featurelist of every replica server contains this feature; so we should dynamically update this "map-like' datastructure in three possible conditions i decribed
   
   **1**
   I think it is still inevitable to add a scheduled timer update-process,   think about this scenerio:  java client fetched an already outdated feature list from  leader metaserver, and this feature list doesn't contains 'BATCH_GET' feature, then java client will  run in old logic; however,  the whole server cluster has been upgraded to new version which support the 'BATCH_GET' feature;   the dilemma is that : java client will not get the `no support` response from the replica because replica also support old-logic-interface, and thus the client will go on running in old logic forever........


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Shuo-Jia commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Shuo-Jia commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-998710744


   
   > Hower the client side may be a little difficult, and even for me it’s a bit vague to what extent the function needs to be implemented, the following pdf is the java client side design I have thought of
   > [feature negotiation client side simple design.pdf](https://github.com/apache/incubator-pegasus/files/7656247/feature.negotiation.client.side.simple.design.pdf)
   
   I notice that the `negotiation` seem to be apply to `meta` and `replica`,   I consider  whether just `negotiation` with `meta` to fetch the feature supporting list. we don't have to handle the different version among `replica`, `meta` can represents the current cluster version. if client request is not supported by replica server, replica server will return error, and triger update `meta` config update. Of course, the meta server need upgrade at last , I think it can be acceptable.
   
   Just consider `meta` maybe let your implement more simple.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1006403703


   
   
   
   > > Hower the client side may be a little difficult, and even for me it’s a bit vague to what extent the function needs to be implemented, the following pdf is the java client side design I have thought of
   > > [feature negotiation client side simple design.pdf](https://github.com/apache/incubator-pegasus/files/7656247/feature.negotiation.client.side.simple.design.pdf)
   > 
   > I notice that the `negotiation` seem to be apply to `meta` and `replica`, I consider whether just `negotiation` with `meta` to fetch the feature supporting list. we don't have to handle the different version among `replica`, `meta` can represents the current cluster version. if client request is not supported by replica server, replica server will return error, and triger update `meta` config update. Of course, the meta server need upgrade at last , I think it can be acceptable.
   > 
   > Just consider `meta` maybe let your implement more simple.
   
   Agree, client fetch "feature map" from meta server would be more simple, and also reduce rpcs between client and servers. We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'. And it's reasonable for MS to know RS 'feature map'.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018391927


   > ## 1
   > > The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically...
   > 
   > Refer to implementation `config update`, you just update after the response of replica is `no support`. rather than boot one scheduled task
   > 
   > ## 2
   > > The server implementation is more complicated and needs to be well tested
   > 
   > You still need implement same or more code on server side when you use the old design which client negotiation replica server, and the new design just apply the logic to meta and you can re-use `config-sync`, so I think it shouldn't be `more complicated`.
   > 
   > You list the update case in
   > 
   > > according to three possible situations
   > 
   > Actually, you may not consider these different case, the `config-sync` just `run` and store the latest value.
   > 
   > Just like @acelyc111 say:
   > 
   > > We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'
   > 
   > ## 3
   > > so the client can only start a periodically scheduled task
   > 
   > As above, refer to implementation of `meta config` , you don't need periodically scheduled task.
   > 
   > ## 4
   > > but the business code of BATCH_GET rpc is in the pegaus
   > 
   > They are two feature, and locate different git-rep is acceptable, it will be improved after they are combined
   **1**
   I think it is still inevitable to add a scheduled timer update-process, think about this scenerio: java client fetched an already outdated feature list from leader metaserver, and this feature list doesn't contains 'BATCH_GET' feature, then java client will run in old logic; however, the whole server cluster has been upgraded to new version which support the 'BATCH_GET' feature; the dilemma is that : java client will not get the no support response from the replica because replica also support old-logic-interface, and thus the client will go on running in old logic forever........


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Shuo-Jia commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Shuo-Jia commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018369154


   ## 1
   > The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically...
   
   Refer to implementation `config update`,  you just update after the response of replica is `no support`. rather than boot one scheduled task
   
   ## 2
   > The server implementation is more complicated and needs to be well tested
   
   You still need implement same or more code on server side when you use the old design which client negotiation replica server, and the new design just apply the logic to meta and you can re-use `config-sync`, so I think it shouldn't be `more complicated`. 
   
   You list the update case in 
   > according to three possible situations
   
   Actually, you may not consider these different case,  the `config-sync` just `run` and store the latest value. 
   
   Just like @acelyc111 say: 
   
   > We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'
   
   
   ## 3
   
   > so the client can only start a periodically scheduled task
   
   As above, refer to implementation of `meta config` , you don't need periodically scheduled task.
   
   ## 4
   > but the business code of BATCH_GET rpc is in the pegaus
   
   They are two feature, and locate different git-rep is acceptable, it will be improved after they are combined
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
hycdong commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-998428752


   That's a good idea to use `feature_map` rather than a version to do client negotiation~
   
   In current implmenetaion, client will firstly create connection with meta server, and get partition address map, then send read/write requests to replica server directly. I notice that the `negotiate_feature_flag` is added in rrdb.thrift according to https://github.com/cauchy1988/incubator-pegasus/pull/1/files, which means client should firstly connect to meta server, then send negotiate request to replica server, if the negotitate failed, client should do lots of thems such as downgrade batchget3 into batchget2 and other consistent-related error handling. 
   
   It seems that the negotiation is designed especially for the batchget3 interface or user request. It can't handle any server request negotiation. In my view, a general negotitation plan should handle user request and server ddl request. For example, client sends negotiation request to meta server, if the operation is not supported, it will reject the connection of this client. Then client could choose to downgrade or do operation. I wonder if you have any plan about to implement a more general negotitation plan?
   
   Besides, I have the following compatible questions:
   1. old client -> new server, will have any compatible problem?
   2. new client -> old server, how old server handle `BATCH_GET` code? will have any compatible problem?
   3. old server upgrade to new server, will it have any compatible problem?
   
   At the end, I DO think your issue is a good idea to solve user request version problems, and expecting your anwser and discussion~


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1006312086


   > It seems that the negotiation is designed especially for the batchget3 interface or
   
   My idea is: the new server itself should be compatible with the old client, this is a common practice in the industry;
   in this issue I focus on solving the problem of compatibility with the old server when the new client develops new functions;
   Then answer your three questions above:
   1. old client-->new server: The new server itself needs to be compatible with the old server as mentioned above, so this is not a problem
   2. new client --> old server: as the method of this issue, downgraded to the old way of use
   3. Old server upgrade to new server: firstly, there is no problem with the old client continuing to use. secondly , the new client will follow the client's implementation plan in this issue: it will ensure that all downstream services support the new feature before sending the rpc of the new protocol  Request, otherwise use the old


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Smityz commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-986469447


   It's a good design, but why not use `version` to determine which implementation to use


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 removed a comment on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 removed a comment on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018390994


   > > ## 1
   > > > The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically...
   > > 
   > > 
   > > Refer to implementation `config update`, you just update after the response of replica is `no support`. rather than boot one scheduled task
   > > ## 2
   > > > The server implementation is more complicated and needs to be well tested
   > > 
   > > 
   > > You still need implement same or more code on server side when you use the old design which client negotiation replica server, and the new design just apply the logic to meta and you can re-use `config-sync`, so I think it shouldn't be `more complicated`.
   > > You list the update case in
   > > > according to three possible situations
   > > 
   > > 
   > > Actually, you may not consider these different case, the `config-sync` just `run` and store the latest value.
   > > Just like @acelyc111 say:
   > > > We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'
   > > 
   > > 
   > > ## 3
   > > > so the client can only start a periodically scheduled task
   > > 
   > > 
   > > As above, refer to implementation of `meta config` , you don't need periodically scheduled task.
   > > ## 4
   > > > but the business code of BATCH_GET rpc is in the pegaus
   > > 
   > > 
   > > They are two feature, and locate different git-rep is acceptable, it will be improved after they are combined
   > 
   > **2** meta server should remember the feature list of all the replica server in a 'map like' datastructure, and judge one feature is supported only when all featurelist of every replica server contains this feature; so we should dynamically update this "map-like' datastructure in three possible conditions i decribed
   
   **1**
   I think it is still inevitable to add a scheduled timer update-process,   think about this scenerio:  java client fetched an already outdated feature list from  leader metaserver, and this feature list doesn't contains 'BATCH_GET' feature, then java client will  run in old logic; however,  the whole server cluster has been upgraded to new version which support the 'BATCH_GET' feature;   the dilemma is that : java client will not get the `no support` response from the replica because replica also support old-logic-interface, and thus the client will go on running in old logic forever........


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 removed a comment on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 removed a comment on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018391927


   > ## 1
   > > The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically...
   > 
   > Refer to implementation `config update`, you just update after the response of replica is `no support`. rather than boot one scheduled task
   > 
   > ## 2
   > > The server implementation is more complicated and needs to be well tested
   > 
   > You still need implement same or more code on server side when you use the old design which client negotiation replica server, and the new design just apply the logic to meta and you can re-use `config-sync`, so I think it shouldn't be `more complicated`.
   > 
   > You list the update case in
   > 
   > > according to three possible situations
   > 
   > Actually, you may not consider these different case, the `config-sync` just `run` and store the latest value.
   > 
   > Just like @acelyc111 say:
   > 
   > > We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'
   > 
   > ## 3
   > > so the client can only start a periodically scheduled task
   > 
   > As above, refer to implementation of `meta config` , you don't need periodically scheduled task.
   > 
   > ## 4
   > > but the business code of BATCH_GET rpc is in the pegaus
   > 
   > They are two feature, and locate different git-rep is acceptable, it will be improved after they are combined
   **1**
   I think it is still inevitable to add a scheduled timer update-process, think about this scenerio: java client fetched an already outdated feature list from leader metaserver, and this feature list doesn't contains 'BATCH_GET' feature, then java client will run in old logic; however, the whole server cluster has been upgraded to new version which support the 'BATCH_GET' feature; the dilemma is that : java client will not get the no support response from the replica because replica also support old-logic-interface, and thus the client will go on running in old logic forever........


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1019293277


   > > java client will not get the no support response
   > 
   > Yeah, I see, you are right
   
   so sadly, I should implement a periodical running task to update the newest feature list from the leader meta server,  can have any better way?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1018384900


   > ## 1
   > > The java client may pull outdated feature list data, so we need a timer repeated scheduled function process used to continuously update the local feature list periodically...
   > 
   > Refer to implementation `config update`, you just update after the response of replica is `no support`. rather than boot one scheduled task
   > 
   > ## 2
   > > The server implementation is more complicated and needs to be well tested
   > 
   > You still need implement same or more code on server side when you use the old design which client negotiation replica server, and the new design just apply the logic to meta and you can re-use `config-sync`, so I think it shouldn't be `more complicated`.
   > 
   > You list the update case in
   > 
   > > according to three possible situations
   > 
   > Actually, you may not consider these different case, the `config-sync` just `run` and store the latest value.
   > 
   > Just like @acelyc111 say:
   > 
   > > We can extend 'config_sync' betwen MS and RS to include replica server's 'feature map'
   > 
   > ## 3
   > > so the client can only start a periodically scheduled task
   > 
   > As above, refer to implementation of `meta config` , you don't need periodically scheduled task.
   > 
   > ## 4
   > > but the business code of BATCH_GET rpc is in the pegaus
   > 
   > They are two feature, and locate different git-rep is acceptable, it will be improved after they are combined
   
   **2**
   meta server should remember the feature list of all the replica server in a 'map like' datastructure, and judge one feature is supported only when all featurelist of every replica server contains this feature;    so we should dynamically update this "map-like' datastructure  in three possible conditions i decribed 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Shuo-Jia commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Shuo-Jia commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1022786934


   > > > java client will not get the no support response
   > > 
   > > 
   > > Yeah, I see, you are right
   > 
   > so sadly, I should implement a periodical running task to update the newest feature list from the leader meta server, can have any better way?
   
   Ok, it's feasible


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-975070188


   I think this is a common use case, not only in your company's case.
   Server side version and client side version may not upgrade synchronized in real world, server may be newer than client, and client may also be newer than server.
   And, some features may be introduced since some version, and it's possible to remove some features since some version.
   Do such a feature based negotiation is reasonable.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 edited a comment on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 edited a comment on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1006312086


   > It seems that the negotiation is designed especially for the batchget3 interface or
   
   My idea is: the new server itself should be compatible with the old client, this is a common practice in the industry;
   in this issue I focus on solving the problem of compatibility with the old server when the new client develops new functions;
   Then answer your three questions above:
   1. old client-->new server: The new server itself needs to be compatible with the old client as mentioned above, so this is not a problem
   2. new client --> old server: as the method of this issue, downgraded to the old way of use
   3. Old server upgrade to new server: firstly, there is no problem with the old client continuing to use. secondly , the new client will follow the client's implementation plan in this issue: it will ensure that all downstream services support the new feature before sending the rpc of the new protocol  Request, otherwise use the old


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] cauchy1988 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
cauchy1988 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1006475468


   > If you still want to handle the replica version, you can let meta to sync the replica server in `config_sync` and return the client.
   > 
   > In short, I think it will be clearer if you only deal with the interaction with meta
   
   I'll implement this issue in this way; @acelyc111 @hycdong @Smityz @Shuo-Jia  Any of you have any comments?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-1006394525


   > It's a good design, but why not use `version` to determine which implementation to use
   
   Some features may be introduced since some version, and it's possible to remove some features since some version, it's diffcult and complex to maintainace such a mapping.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Shuo-Jia commented on issue #840: add 'feature flags' rcp interface for version compatability between server and client

Posted by GitBox <gi...@apache.org>.
Shuo-Jia commented on issue #840:
URL: https://github.com/apache/incubator-pegasus/issues/840#issuecomment-977702481


   > I think this is a common use case, not only in your company's case. Server side version and client side version may not upgrade synchronized in real world, server may be newer than client, and client may also be newer than server. And, some features may be introduced since some version, and it's possible to remove some features since some version. Do such a feature based negotiation is reasonable.
   
   +1, the future is necessary


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org