You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2020/09/05 12:37:45 UTC

[GitHub] [shardingsphere] dongfangding opened a new issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

dongfangding opened a new issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269


   ## The question about dataSourceNames only taking the last one after master-slave-rules is configured
   
   Please forgive me, I have to use Google Translate due to the limitation of English only.
   
   In the process of my study, I configured one master and multiple slaves. One of the tables is used for master-slave synchronization + read-write separation, and then there is another table I use for horizontal sub-database sub-table. Then I configured `master-slave-rules`, but I received the error `Cannot find data source in sharding rule, invalid actual data node is:'slave0.user_article'`.
   I had to look at some code to find out if there was a problem with my configuration, and then I found some confusing code.
   `org.apache.shardingsphere.core.rule.ShardingDataSourceNames#getAllDataSourceNames`
   ```java
       private Collection<String> getAllDataSourceNames(final Collection<String> dataSourceNames) {
           Collection<String> result = new LinkedHashSet<>(dataSourceNames);
           for (MasterSlaveRuleConfiguration each : shardingRuleConfig.getMasterSlaveRuleConfigs()) {
               result.remove(each.getMasterDataSourceName());
               result.removeAll(each.getSlaveDataSourceNames());
               result.add(each.getName());
           }
           return result;
       }
   ```
   
   It looks like if I configure `master-slave-rules`, `dataSourceNames` will only keep the last one, but I have a table that does not need to be read-write separated, so my `actual-data-nodes` Contains the slave datasource name I want. then
   I was blocked by the following judgments, and I could not start my project.
   
   `org.apache.shardingsphere.core.rule.TableRule#generateDataNodes(java.lang.String, java.util.Collection<java.lang.String>)`
   ```java
       private List<DataNode> generateDataNodes(final List<String> actualDataNodes, final Collection<String> dataSourceNames) {
           List<DataNode> result = new LinkedList<>();
           int index = 0;
           for (String each : actualDataNodes) {
               DataNode dataNode = new DataNode(each);
               if (!dataSourceNames.contains(dataNode.getDataSourceName())) {
                   throw new ShardingSphereException("Cannot find data source in sharding rule, invalid actual data node is: '%s'", each);
               }
               result.add(dataNode);
               dataNodeIndexMap.put(dataNode, index);
               actualDatasourceNames.add(dataNode.getDataSourceName());
               addActualTable(dataNode.getDataSourceName(), dataNode.getTableName());
               index++;
           }
           return result;
       }
   ```
   
   Then, I tried to modify part of the code. Because I am not familiar with this project, I am very careful to provide an attribute in `YamlTableRuleConfiguration` so that I can decide to pass in the master datasource name of `master-slave-rule` configuration when `new TableRule()` , Or pass in all configured dataSourceNames, and finally I found a solution to the problem.
   
   Then I made a bold change, returning all dataSourceNames directly in `getAllDataSourceNames`, and then I found that everything still works.
   
   I am currently using the latest version 4.1.1, because I need to solve this problem now, please also provide an idea, can I use the above two methods to solve my problem.
   
   Of course, in the latest code, I found that this problem seems to have been modified, but the entire project code seems to have changed a lot, and there is no version you provided that can be used directly, so I still have to try to solve my problem locally.
   
   I would be very grateful if you could impatiently finish reading the wordy above, and I am also deeply sorry if you waste your precious time.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688590710


   Hi @dongfangding 
   Could you try to remove `slave$->{0..1}.user_article,slave$->{0..1}.user_article_1` and restart again?


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688651647


   It would be ok, but it's not what i want.
   As I said above,  **It looks like if I configure master-slave-rules, dataSourceNames will only keep the last one. but I have a table that does not need to be read-write separated, so my actual-data-nodes Contains the slave datasource name I want.**
   
   The tables I am talking about are `user_article` and `user_article_1`, so i configured `actual-data-nodes: master.user_article,master.user_article_1,slave$->{0..1}.user_article,slave$->{0..1}.user_article_1`.
   
   According to my current understanding, if the actual internal logic is normal, it can support my current needs, and the only problem lies in the judgment, that is, the title of this issu, which blocked my thoughts.
   
   Let me add that I expect to sub-database the logical table `user_article` based on `user_id`. After I configured the master-slave-rule, I will ignore the slave synchronization action for user_article in my.conf.
   
   ```java
   public class UserArticleDatabasePreciseShardingAlgorithm implements PreciseShardingAlgorithm<Long> {
   
       /**
        * Sharding.
        *
        * @param availableTargetNames available data sources or tables's names
        * @param shardingValue        sharding value
        * @return sharding result for data source or table's name
        */
       @Override
       public String doSharding(Collection<String> availableTargetNames, PreciseShardingValue<Long> shardingValue) {
           if (CollUtil.isEmpty(availableTargetNames)) {
               throw new RuntimeException("无可用的数据源");
           }
           // I want availableTargetNames include master,slave0,slave1, so i configured actual-data-nodes: master.user_article,master.user_article_1,slave$->{0..1}.user_article,slave$->{0..1}.user_article_1
           Long value = shardingValue.getValue();
           long index = value % availableTargetNames.size();
   
           int i = 0;
           for (String availableTargetName : availableTargetNames) {
               if (i == index) {
                   return availableTargetName;
               }
               i ++;
           }
           return null;
       }
   }
   ```
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-691598089


   > # If i want to apply the master-slave-rule, does this correspond to the datasourcename in the master-slaves-rule?
   Yes.
   
   > # But here, I don’t want to apply the master-slave-rule, I only need to keep the name of the original data source consistent, right?
   Yes.
   
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690343941


   There seems to be no difference, I renamed master to master01, but it is still not the result I want.
   
   sharding-column user_id ==> Its value is 99.
   When it is assigned to master datasource, The query statement still follows the master-slave-rule.
   ```
   Actual SQL: slave1 ::: SELECT..
   Actual SQL: master01 ::: INSERT INTO user_article_1
   ```
   
   I have to debug the code again. I just want to know if there is a statement about the questionable code I mentioned earlier. It's like I'm already discussing routing issues, but in fact, my current configuration, if I don't modify part of the source code, the current project cannot be started. The current problem is that I still have routing problems after I start it. So I have to ask, is there a problem with my use, or the configuration I mentioned earlier is not supported by itself.
   
   Now I have to extract part of the code again, which makes me feel that my configuration is not supported at present.
   
   There are two RouteDecorators mentioned earlier. When I enter this code for the second time, if the route of my sub-database happens to be the master at this time, then it will apply the master-slave-rule for me.
   
   org.apache.shardingsphere.masterslave.route.engine.MasterSlaveRouteDecorator#decorate
   ```java
       @Override
       public RouteContext decorate(final RouteContext routeContext, final ShardingSphereMetaData metaData, final MasterSlaveRule masterSlaveRule, final ConfigurationProperties properties) {
           if (routeContext.getRouteResult().getRouteUnits().isEmpty()) {
               String dataSourceName = new MasterSlaveDataSourceRouter(masterSlaveRule).route(routeContext.getSqlStatementContext().getSqlStatement());
               RouteResult routeResult = new RouteResult();
               routeResult.getRouteUnits().add(new RouteUnit(new RouteMapper(dataSourceName, dataSourceName), Collections.emptyList()));
               return new RouteContext(routeContext.getSqlStatementContext(), Collections.emptyList(), routeResult);
           }
           Collection<RouteUnit> toBeRemoved = new LinkedList<>();
           Collection<RouteUnit> toBeAdded = new LinkedList<>();
           for (RouteUnit each : routeContext.getRouteResult().getRouteUnits()) {
               if (masterSlaveRule.getName().equalsIgnoreCase(each.getDataSourceMapper().getActualName())) {
                   toBeRemoved.add(each);
                   String actualDataSourceName = new MasterSlaveDataSourceRouter(masterSlaveRule).route(routeContext.getSqlStatementContext().getSqlStatement());
                   toBeAdded.add(new RouteUnit(new RouteMapper(each.getDataSourceMapper().getLogicName(), actualDataSourceName), each.getTableMappers()));
               }
           }
           routeContext.getRouteResult().getRouteUnits().removeAll(toBeRemoved);
           routeContext.getRouteResult().getRouteUnits().addAll(toBeAdded);
           return routeContext;
       }
   ```
   
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] terrymanu commented on issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
terrymanu commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-696640580


   Your configuration is not the best practice of horizontal sharding, it is better to avoid to spent lots of time do develop a rare requirement.
   I just close it, please try to use normal horizontal sharding configuration.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690858993


   Hi @dongfangding 
   
   Let us make it clear.
   I assume that you can still configure any of the `slaves` or `masters` node for any sharding table even though there is master-slave rule.
   However, your statement told me that this `master-slave` node rather than any slaves from this `master-slave` node would be used, as long as `master-slave` rule exists.
   
   **Is that our conclusion now?**
   
   Can you make sure the following configuration is exactly what you pointed?
   Please pay attention to the `master-slave` rule and `actualDataNodes` for `t_order_item`.
   
   ```yaml
   // A configuration for ShardingProzy
   
   schemaName: sharding_db
   
   dataSourceCommon:
     username: root
     password:
     connectionTimeoutMilliseconds: 30000
     idleTimeoutMilliseconds: 60000
     maxLifetimeMilliseconds: 1800000
     maxPoolSize: 50
     minPoolSize: 1
     maintenanceIntervalMilliseconds: 30000
   
   dataSources:
     ds_master_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_0_slave_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_0_slave_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
     ds_master_1_slave_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
     ds_master_1_slave_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
   
   rules:
     - !SHARDING
       tables:
         t_order:
           actualDataNodes: ds_${0..1}.t_order_${0..1}
           tableStrategy:
             standard:
               shardingColumn: order_id
               shardingAlgorithmName: t_order_inline
           keyGenerateStrategy:
             column: order_id
             keyGeneratorName: snowflake
         t_order_item:
           actualDataNodes: ds_master_1_slave_${0..1}.t_order_item_${0..1}
           tableStrategy:
             standard:
               shardingColumn: order_id
               shardingAlgorithmName: t_order_item_inline
           keyGenerateStrategy:
             column: order_id
             keyGeneratorName: snowflake
       bindingTables:
         - t_order,t_order_item
       broadcastTables:
         - t_address
       defaultDatabaseStrategy:
         standard:
           shardingColumn: user_id
           shardingAlgorithmName: database_inline
   
       shardingAlgorithms:
         database_inline:
           type: INLINE
           props:
             algorithm-expression: ds_${user_id % 2}
         t_order_inline:
           type: INLINE
           props:
             algorithm-expression: t_order_${order_id % 2}
         t_order_item_inline:
           type: INLINE
           props:
             algorithm-expression: t_order_item_${order_id % 2}
   
       keyGenerators:
         snowflake:
           type: SNOWFLAKE
           props:
             worker-id: 123
   
     - !MASTER_SLAVE
       dataSources:
         ds_0:
           masterDataSourceName: ds_master_0
           slaveDataSourceNames: [ds_master_0_slave_0, ds_master_0_slave_1]
           loadBalancerName: roundRobin
         ds_1:
           masterDataSourceName: ds_master_1
           slaveDataSourceNames: [ds_master_1_slave_0, ds_master_1_slave_1]
           loadBalancerName: roundRobin
       loadBalancers:
         roundRobin:
           type: ROUND_ROBIN
   
   props:
     sql.show: false
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-693155452


   Hi @dongfangding thanks for your continued attention.
   
   Recently I was working on #7442, which touched your issue somewhat. At that time, I realized what you mentioned is not an easy one, which is related to how we treat our actual data sources and `master-salve` data sources.
   
   Please leave us some time to answer your question. I will come back once the discussion finishes. :-)
   
   Best,
   Trista


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-689273122


   Do you mean like this ?    
   ```yaml
     master-slave-rules:
           myMaster:
             load-balance-algorithm-type: round_robin
             master-data-source-name: master
             slave-data-source-names: slave0,slave1
   ```
   It will cause me to be unable to start the project, so can you provide me with the specific configuration you mentioned
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690858993


   Hi @dongfangding 
   
   Let us make it clear.
   I assume that you can still configure any of the `slaves` or `masters` node for any sharding table even though there is master-slave rule.
   However, your statement told me that this `master-slave` node rather than any slaves from this `master-slave` node would be used, as long as `master-slave` rule exists.
   
   **Is that our conclusion now?**
   
   Can you make sure the following configuration is exactly what you pointed?
   Please pay attention to the `master-slave` rule and `actualDataNodes` for `t_order_item`.
   
   ```yaml
   // A configuration for ShardingProzy
   
   schemaName: sharding_db
   
   dataSourceCommon:
     username: root
     password:
     connectionTimeoutMilliseconds: 30000
     idleTimeoutMilliseconds: 60000
     maxLifetimeMilliseconds: 1800000
     maxPoolSize: 50
     minPoolSize: 1
     maintenanceIntervalMilliseconds: 30000
   
   dataSources:
     ds_master_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_0_slave_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_0_slave_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
     ds_master_1_slave_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
     ds_master_1_slave_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
   
   rules:
     - !SHARDING
       tables:
         t_order:
           actualDataNodes: ds_${0..1}.t_order_${0..1}
           tableStrategy:
             standard:
               shardingColumn: order_id
               shardingAlgorithmName: t_order_inline
           keyGenerateStrategy:
             column: order_id
             keyGeneratorName: snowflake
         t_order_item:
           actualDataNodes: ds_master_1_slave_${0..1}.t_order_item_${0..1}
           tableStrategy:
             standard:
               shardingColumn: order_id
               shardingAlgorithmName: t_order_item_inline
           keyGenerateStrategy:
             column: order_id
             keyGeneratorName: snowflake
       bindingTables:
         - t_order,t_order_item
       broadcastTables:
         - t_address
       defaultDatabaseStrategy:
         standard:
           shardingColumn: user_id
           shardingAlgorithmName: database_inline
   
       shardingAlgorithms:
         database_inline:
           type: INLINE
           props:
             algorithm-expression: ds_${user_id % 2}
         t_order_inline:
           type: INLINE
           props:
             algorithm-expression: t_order_${order_id % 2}
         t_order_item_inline:
           type: INLINE
           props:
             algorithm-expression: t_order_item_${order_id % 2}
   
       keyGenerators:
         snowflake:
           type: SNOWFLAKE
           props:
             worker-id: 123
   
     - !MASTER_SLAVE
       dataSources:
         ds_0:
           masterDataSourceName: ds_master_0
           slaveDataSourceNames: [ds_master_0_slave_0, ds_master_0_slave_1]
           loadBalancerName: roundRobin
         ds_1:
           masterDataSourceName: ds_master_1
           slaveDataSourceNames: [ds_master_1_slave_0, ds_master_1_slave_1]
           loadBalancerName: roundRobin
       loadBalancers:
         roundRobin:
           type: ROUND_ROBIN
   
   props:
     sql.show: false
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688235597


   Yes, you are right, but in my case, I think it's necessary.
   
   About configuration, it like this.
   
   ```yaml
       sharding:
         default-data-source-name: master
         tables:
           auth_user:
             actual-data-nodes: master.auth_user,master.auth_user_1
             table-strategy:
               inline:
                 sharding-column: id
                 algorithm-expression: auth_user$->{id % 2 == 0 ? "":"_" + id % 2}
   
           user_article:
             userMasterSlaveRulesDatasourceNameIfExist: false
             actual-data-nodes: master.user_article,master.user_article_1,slave$->{0..1}.user_article,slave$->{0..1}.user_article_1
             database-strategy:
               standard:
                 sharding-column: user_id
                 preciseAlgorithmClassName: com.ddf.boot.quick.sharding.UserArticleDatabasePreciseShardingAlgorithm
   
             table-strategy:
               inline:
                 sharding-column: user_id
                 algorithm-expression: user_article$->{user_id % 2 == 0 ? "":"_" + user_id % 2}
   
         master-slave-rules:
           master:
             load-balance-algorithm-type: round_robin
             master-data-source-name: master
             slave-data-source-names: slave0,slave1
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690343941


   There seems to be no difference, I renamed master to master01, but it is still not the result I want.
   
   sharding-column user_id ==> Its value is 99.
   When it is assigned to master datasource, The query statement still follows the master-slave-rule.
   ```
   Actual SQL: slave1 ::: SELECT..
   Actual SQL: master01 ::: INSERT INTO user_article_1
   ```
   
   I have to debug the code again. I just want to know if there is a statement about the questionable code I mentioned earlier. It's like I'm already discussing routing issues, but in fact, my current configuration, if I don't modify part of the source code, the current project cannot be started. The current problem is that I still have routing problems after I start it. So I have to ask, is there a problem with my use, or the configuration I mentioned earlier is not supported by itself.
   
   Now I have to extract part of the code again, which makes me feel that my configuration is not supported at present.
   
   There are two RouteDecorators mentioned earlier. When I enter this code for the second time, if the route of my sub-database happens to be the master at this time, then it will apply the master-slave-rule for me.
   
   ```java
       @Override
       public RouteContext decorate(final RouteContext routeContext, final ShardingSphereMetaData metaData, final MasterSlaveRule masterSlaveRule, final ConfigurationProperties properties) {
           if (routeContext.getRouteResult().getRouteUnits().isEmpty()) {
               String dataSourceName = new MasterSlaveDataSourceRouter(masterSlaveRule).route(routeContext.getSqlStatementContext().getSqlStatement());
               RouteResult routeResult = new RouteResult();
               routeResult.getRouteUnits().add(new RouteUnit(new RouteMapper(dataSourceName, dataSourceName), Collections.emptyList()));
               return new RouteContext(routeContext.getSqlStatementContext(), Collections.emptyList(), routeResult);
           }
           Collection<RouteUnit> toBeRemoved = new LinkedList<>();
           Collection<RouteUnit> toBeAdded = new LinkedList<>();
           for (RouteUnit each : routeContext.getRouteResult().getRouteUnits()) {
               if (masterSlaveRule.getName().equalsIgnoreCase(each.getDataSourceMapper().getActualName())) {
                   toBeRemoved.add(each);
                   String actualDataSourceName = new MasterSlaveDataSourceRouter(masterSlaveRule).route(routeContext.getSqlStatementContext().getSqlStatement());
                   toBeAdded.add(new RouteUnit(new RouteMapper(each.getDataSourceMapper().getLogicName(), actualDataSourceName), each.getTableMappers()));
               }
           }
           routeContext.getRouteResult().getRouteUnits().removeAll(toBeRemoved);
           routeContext.getRouteResult().getRouteUnits().addAll(toBeAdded);
           return routeContext;
       }
   ```
   
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-698739771


   > I only want to sub-database with that individual table
   
   That is still a type of Sharding strategy.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-691433175


   I got some inspiration and adjusted the actual-data-node of auth_user and user_article.
   But I don't have that many data sources. If I only have three data sources, I didn't consider `database-strategy` initially, so this is enough for me.
   But now I only hope that one of the tables has `database-strategy`, that is `user_article`, and the others are still `master-slave-rule`, can it only be configured like I am now? So does it support it? Because I went back to my original question,
    If I configure master-slave-rules, then the logic-databasename in `actual-data-nodes` must exist in master-slave-rule?
   
   The happy thing. After I solved this problem, it now looks normal.
   
   ```yaml
   shardingsphere:
       datasource:
         names: master,slave0,slave1
         master:
         slave0:
         slave1:  
       sharding:
         default-data-source-name: master
         tables:
           auth_user:
            # If i want to apply the master-slave-rule, does this correspond to the datasourcename in the master-slaves-rule?
             actual-data-nodes: master1.auth_user,master1.auth_user_1
             table-strategy:
               inline:
                 sharding-column: id            
                 algorithm-expression: auth_user$->{id % 2 == 0 ? "":"_" + id % 2}
   
           user_article:
             # But here, I don’t want to apply the master-slave-rule, I only need to keep the name of the original data source consistent, right?
             actual-data-nodes: master.user_article,master.user_article_1,slave$->{0..1}.user_article,slave$->{0..1}.user_article_1
             database-strategy:
               standard:
                 sharding-column: user_id
                 preciseAlgorithmClassName: com.ddf.boot.quick.sharding.UserArticleDatabasePreciseShardingAlgorithm
   
             table-strategy:
               inline:
                 sharding-column: user_id
                 algorithm-expression: user_article$->{user_id % 2 == 0 ? "":"_" + user_id % 2}
   
         master-slave-rules:
           master1:
             load-balance-algorithm-type: round_robin
             master-data-source-name: master
             slave-data-source-names: slave0,slave1
       props:
         sql.show: true
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688231173


   Hi @dongfangding 
   Thanks for your question and for debugging in the source code. Could you provide your configuration?
   As you said, I supposed you configured `sharding rule` and `master-slave rule` together, didn't you?


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690858993


   Hi @dongfangding 
   
   Let us make it clear.
   I assume that you can still configure any of the `slaves` or `masters` node for any sharding table even though there is master-slave rule.
   However, your statement told me that this `master-slave` node rather than any slaves from this `master-slave` node would be used, as long as `master-slave` rule exists.
   Does that is our conclusion?
   
   Can you make sure the following configuration is exactly what you pointed?
   Please pay attention to the `master-slave` rule and `actualDataNodes` for `t_order_item`.
   
   ```yaml
   // A configuration for ShardingProzy
   
   schemaName: sharding_db
   
   dataSourceCommon:
     username: root
     password:
     connectionTimeoutMilliseconds: 30000
     idleTimeoutMilliseconds: 60000
     maxLifetimeMilliseconds: 1800000
     maxPoolSize: 50
     minPoolSize: 1
     maintenanceIntervalMilliseconds: 30000
   
   dataSources:
     ds_master_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_0_slave_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_0_slave_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
     ds_master_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
     ds_master_1_slave_0:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
     ds_master_1_slave_1:
       url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
   
   rules:
     - !SHARDING
       tables:
         t_order:
           actualDataNodes: ds_${0..1}.t_order_${0..1}
           tableStrategy:
             standard:
               shardingColumn: order_id
               shardingAlgorithmName: t_order_inline
           keyGenerateStrategy:
             column: order_id
             keyGeneratorName: snowflake
         t_order_item:
           actualDataNodes: ds_master_1_slave_${0..1}.t_order_item_${0..1}
           tableStrategy:
             standard:
               shardingColumn: order_id
               shardingAlgorithmName: t_order_item_inline
           keyGenerateStrategy:
             column: order_id
             keyGeneratorName: snowflake
       bindingTables:
         - t_order,t_order_item
       broadcastTables:
         - t_address
       defaultDatabaseStrategy:
         standard:
           shardingColumn: user_id
           shardingAlgorithmName: database_inline
   
       shardingAlgorithms:
         database_inline:
           type: INLINE
           props:
             algorithm-expression: ds_${user_id % 2}
         t_order_inline:
           type: INLINE
           props:
             algorithm-expression: t_order_${order_id % 2}
         t_order_item_inline:
           type: INLINE
           props:
             algorithm-expression: t_order_item_${order_id % 2}
   
       keyGenerators:
         snowflake:
           type: SNOWFLAKE
           props:
             worker-id: 123
   
     - !MASTER_SLAVE
       dataSources:
         ds_0:
           masterDataSourceName: ds_master_0
           slaveDataSourceNames: [ds_master_0_slave_0, ds_master_0_slave_1]
           loadBalancerName: roundRobin
         ds_1:
           masterDataSourceName: ds_master_1
           slaveDataSourceNames: [ds_master_1_slave_0, ds_master_1_slave_1]
           loadBalancerName: roundRobin
       loadBalancers:
         roundRobin:
           type: ROUND_ROBIN
   
   props:
     sql.show: false
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-691598089


   >  If i want to apply the master-slave-rule, does this correspond to the datasourcename in the master-slaves-rule?
   Yes.
   
   >  But here, I don’t want to apply the master-slave-rule, I only need to keep the name of the original data source consistent, right?
   Yes.
   
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-692460469


   In the current version, the current configuration will not start the project. This is also the original intention of my issue. 
   
    I configure `master-slave-rules`, The logic-datasource-name contained in `actual-data-nodes` seems to have to exist in the master-salve-rule.
   
   I am not sure.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-691433175


   I got some inspiration and adjusted the actual-data-node of auth_user and user_article.
   But I don't have that many data sources. If I only have three data sources, I didn't consider `database-strategy` initially, so this is enough for me.
   But now I only hope that one of the tables has `database-strategy`, that is `user_article`, and the others are still `master-slave-rule`, can it only be configured like I am now? So does it support it? Because I went back to my original question,
    If I configure master-slave-rules, then the logic-databasename in `actual-data-nodes` must exist in master-slave-rule?
   
   The happy thing. After I solved this problem, it now looks normal.
   
   ```yaml
   shardingsphere:
       datasource:
         names: master,slave0,slave1
         master:
         slave0:
         slave1:  
       sharding:
         default-data-source-name: master
         tables:
           auth_user:
            # If i want to apply the master-slave-rule, does this correspond to the datasourcename in the master-slaves-rule?
             actual-data-nodes: master1.auth_user,master1.auth_user_1
             table-strategy:
               inline:
                 sharding-column: id            
                 algorithm-expression: auth_user$->{id % 2 == 0 ? "":"_" + id % 2}
   
           user_article:
             # But here, I don’t want to apply the master-slave-rule, I only need to keep the name of the original data source consistent, right?
             actual-data-nodes: master.user_article,master.user_article_1,slave$->{0..1}.user_article,slave$->{0..1}.user_article_1
             database-strategy:
               standard:
                 sharding-column: user_id
                 preciseAlgorithmClassName: com.ddf.boot.quick.sharding.UserArticleDatabasePreciseShardingAlgorithm
   
             table-strategy:
               inline:
                 sharding-column: user_id
                 algorithm-expression: user_article$->{user_id % 2 == 0 ? "":"_" + user_id % 2}
   
         master-slave-rules:
           master1:
             load-balance-algorithm-type: round_robin
             master-data-source-name: master
             slave-data-source-names: slave0,slave1
       props:
         sql.show: true
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-692460469


   In the current version, the current configuration will not start the project. This is also the original intention of my issue. 
   
    If I configured `master-slave-rules`, The logic-datasource-name contained in `actual-data-nodes` seems to have to exist in the master-salve-rule.
   
   I am not sure.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688773969


   I still have to reiterate my scenario, that is, multiple tables, some tables I want to use master-slave-rule, and other tables, I just want it to implement its own TableRule.
   
   Because I didn't pay too much attention to the query before, I found a new problem now. It was a query problem after sub-database. This route did not take effect and was eventually overwritten by master-slave-rule.
   
   I noticed that the program will register two classes, and they have a clear order. 
   * org.apache.shardingsphere.sharding.route.engine.ShardingRouteDecorator
   * class org.apache.shardingsphere.masterslave.route.engine.MasterSlaveRouteDecorator
   
   When I only configure ShardingRule in the project, the routing rules of each table will take effect. And when I configure MasterSlaveRule, due to the order of the Decorator registered earlier, the latter will still cover my control at the table level.
   Like the scenario I mentioned before, this is not what I want, and I am not sure whether this right of determination can be made available to developers.
   
   **org.apache.shardingsphere.underlying.pluggble.prepare.BasePrepareEngine#registerRouteDecorator**
   ```java
   	private void registerRouteDecorator() {
           for (Class<? extends RouteDecorator> each : OrderedRegistry.getRegisteredClasses(RouteDecorator.class)) {
               RouteDecorator routeDecorator = createRouteDecorator(each);
               Class<?> ruleClass = (Class<?>) routeDecorator.getType();
               // FIXME rule.getClass().getSuperclass() == ruleClass for orchestration, should decouple extend between orchestration rule and sharding rule
               rules.stream().filter(rule -> rule.getClass() == ruleClass || rule.getClass().getSuperclass() == ruleClass).collect(Collectors.toList())
                       .forEach(rule -> router.registerDecorator(rule, routeDecorator));
           }
       }
   ```
   
   **org.apache.shardingsphere.underlying.route.DataNodeRouter**
   ```java
   	private RouteContext executeRoute(final String sql, final List<Object> parameters, final boolean useCache) {
           RouteContext result = createRouteContext(sql, parameters, useCache);
           for (Entry<BaseRule, RouteDecorator> entry : decorators.entrySet()) {
               result = entry.getValue().decorate(result, metaData, entry.getKey(), properties);
           }
           return result;
       }
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-698253468


   Sorry, I still have some questions that I need to disturb.
   I'm just a newbie who just got in touch with this one.
   In my demo, I only hoped to do read-write separation at the beginning of the period, and assumed that this is enough for my application, but later I only want to sub-database with that individual table, and the overall architecture I still think read-write separation is Can satisfy me, is this kind of unreasonable existence?
   
   Isn’t this exactly what you stated in the documentation?
   
   https://shardingsphere.apache.org/document/legacy/4.x/document/cn/features/read-write-split/core-features/
   
   1. 提供一主多从的读写分离配置,可独立使用,也可配合分库分表使用。
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-696534740


   Hi @dongfangding 
   
   Sorry for late reply.  As I said before, the masterSlave dataSource is an important subject for ShardingSphere. We will give it a deep consideration and possible refactoring work before 5.x release. Currently, @terrymanu is taking charge of this issue. Any updates will be linked here, then you can follow up on this issue for its progress.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] terrymanu commented on issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
terrymanu commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-696640580


   Your configuration is not the best practice of horizontal sharding, it is better to avoid to spent lots of time do develop a rare requirement.
   I just close it, please try to use normal horizontal sharding configuration.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688773969


   I still have to reiterate my scenario, that is, multiple tables, some tables I want to use master-slave-rule, and other tables, I just want it to implement its own TableRule.
   
   Because I didn't pay too much attention to the query before, I found a new problem now. It was a query problem after sub-database. This route did not take effect and was eventually overwritten by master-slave-rule.
   
   I noticed that the program will register two classes, and they have a clear order. 
   * org.apache.shardingsphere.sharding.route.engine.ShardingRouteDecorator
   * class org.apache.shardingsphere.masterslave.route.engine.MasterSlaveRouteDecorator
   
   When I only configure ShardingRule in the project, the routing rules of each table will take effect. And when I configure MasterSlaveRule, due to the order of the Decorator registered earlier, the latter will still cover my control at the table level.
   Like the scenario I mentioned before, this is not what I want, and I am not sure whether this right of determination can be made available to developers.
   
   **org.apache.shardingsphere.underlying.pluggble.prepare.BasePrepareEngine#registerRouteDecorator**
   ```java
   private void registerRouteDecorator() {
           for (Class<? extends RouteDecorator> each : OrderedRegistry.getRegisteredClasses(RouteDecorator.class)) {
               RouteDecorator routeDecorator = createRouteDecorator(each);
               Class<?> ruleClass = (Class<?>) routeDecorator.getType();
               // FIXME rule.getClass().getSuperclass() == ruleClass for orchestration, should decouple extend between orchestration rule and sharding rule
               rules.stream().filter(rule -> rule.getClass() == ruleClass || rule.getClass().getSuperclass() == ruleClass).collect(Collectors.toList())
                       .forEach(rule -> router.registerDecorator(rule, routeDecorator));
           }
   }
   ```
   
   **org.apache.shardingsphere.underlying.route.DataNodeRouter**
   ```java
   private RouteContext executeRoute(final String sql, final List<Object> parameters, final boolean useCache) {
           RouteContext result = createRouteContext(sql, parameters, useCache);
           for (Entry<BaseRule, RouteDecorator> entry : decorators.entrySet()) {
               result = entry.getValue().decorate(result, metaData, entry.getKey(), properties);
           }
           return result;
   }
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] terrymanu closed issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
terrymanu closed issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269


   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-691598089


   >  If i want to apply the master-slave-rule, does this correspond to the datasourcename in the master-slaves-rule?
   
   Yes.
   
   >  But here, I don’t want to apply the master-slave-rule, I only need to keep the name of the original data source consistent, right?
   
   Yes.
   
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690343941


   There seems to be no difference, I renamed master to master01, but it is still not the result I want.
   
   sharding-column user_id ==> Its value is 99.
   When it is assigned to master datasource, The query statement still follows the master-slave-rule.
   ```
   Actual SQL: slave1 ::: SELECT..
   Actual SQL: master01 ::: INSERT INTO user_article_1
   ```
   
   I have to debug the code again. I just want to know if there is a statement about the questionable code I mentioned earlier. It's like I'm already discussing routing issues, but in fact, my current configuration, if I don't modify part of the source code, the current project cannot be started. The current problem is that I still have routing problems after I start it. So I have to ask, is there a problem with my use, or the configuration I mentioned earlier is not supported by itself.
   
   Now I have to extract part of the code again, which makes me feel that my configuration is not supported at present.
   
   There are two RouteDecorators mentioned earlier. When I enter this code for the second time, if the route of my sub-database happens to be the master at this time, then it will apply the master-slave-rule for me.
   
   `**org.apache.shardingsphere.masterslave.route.engine.MasterSlaveRouteDecorator#decorate**`
   ```java
       @Override
       public RouteContext decorate(final RouteContext routeContext, final ShardingSphereMetaData metaData, final MasterSlaveRule masterSlaveRule, final ConfigurationProperties properties) {
           if (routeContext.getRouteResult().getRouteUnits().isEmpty()) {
               String dataSourceName = new MasterSlaveDataSourceRouter(masterSlaveRule).route(routeContext.getSqlStatementContext().getSqlStatement());
               RouteResult routeResult = new RouteResult();
               routeResult.getRouteUnits().add(new RouteUnit(new RouteMapper(dataSourceName, dataSourceName), Collections.emptyList()));
               return new RouteContext(routeContext.getSqlStatementContext(), Collections.emptyList(), routeResult);
           }
           Collection<RouteUnit> toBeRemoved = new LinkedList<>();
           Collection<RouteUnit> toBeAdded = new LinkedList<>();
           for (RouteUnit each : routeContext.getRouteResult().getRouteUnits()) {
               if (masterSlaveRule.getName().equalsIgnoreCase(each.getDataSourceMapper().getActualName())) {
                   toBeRemoved.add(each);
                   String actualDataSourceName = new MasterSlaveDataSourceRouter(masterSlaveRule).route(routeContext.getSqlStatementContext().getSqlStatement());
                   toBeAdded.add(new RouteUnit(new RouteMapper(each.getDataSourceMapper().getLogicName(), actualDataSourceName), each.getTableMappers()));
               }
           }
           routeContext.getRouteResult().getRouteUnits().removeAll(toBeRemoved);
           routeContext.getRouteResult().getRouteUnits().addAll(toBeAdded);
           return routeContext;
       }
   ```
   
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690858993






----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-698739771


   > I only want to sub-database with that individual table
   
   That is still a type of Sharding strategy.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-689954246


   Yep, that is what I meant. Plus, did you change its reference used together?


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-691433175






----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690858993






----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-689273122


   Do you mean like this ?    
   ```yaml
     master-slave-rules:
           myMaster:
             load-balance-algorithm-type: round_robin
             master-data-source-name: master
             slave-data-source-names: slave0,slave1
   ```
   
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-689273122


   Do you mean like this ?    
   ```yaml
     master-slave-rules:
           myMaster:
             load-balance-algorithm-type: round_robin
             master-data-source-name: master
             slave-data-source-names: slave0,slave1
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-689273122


   Do you mean like this ?    
   ```yaml
     master-slave-rules:
           myMaster:
             load-balance-algorithm-type: round_robin
             master-data-source-name: master
             slave-data-source-names: slave0,slave1
   ```
   
   It's still like this
   
   ```
   Actual SQL: slave1 ::: SELECT...............
   Actual SQL: master ::: INSERT INTO..............
   ```


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688590710


   Hi @dongfangding 
   Could you try to remove `slave$->{0..1}.user_article,slave$->{0..1}.user_article_1` and restart agagin?


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688651647


   It would be ok, but it's not what i want.
   As I said above,  **It looks like if I configure master-slave-rules, dataSourceNames will only keep the last one. but I have a table that does not need to be read-write separated, so my actual-data-nodes Contains the slave datasource name I want.**
   
   The tables I am talking about are `user_article` and `user_article_1`, so i configured `actual-data-nodes: master.user_article,master.user_article_1,slave$->{0..1}.user_article,slave$->{0..1}.user_article_1`.
   
   According to my current understanding, if the actual internal logic is normal, it can support my current needs, and the only problem lies in the judgment, that is, the title of this issu, which blocked my thoughts.
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-688651647


   It would be ok, but it's not what i want.
   As I said above,  **It looks like if I configure master-slave-rules, dataSourceNames will only keep the last one. but I have a table that does not need to be read-write separated, so my actual-data-nodes Contains the slave datasource name I want.**
   
   The tables I am talking about are `user_article` and `user_article_1`, so i configured `actual-data-nodes: master.user_article,master.user_article_1,slave$->{0..1}.user_article,slave$->{0..1}.user_article_1`.
   
   According to my current understanding, if the actual internal logic is normal, it can support my current needs, and the only problem lies in the judgment, that is, the title of this issu, which blocked my thoughts.
   
   Let me add that I expect to sub-database the logical table `user_article` based on `user_id`. After I configured the master-slave-rule, I will ignore the slave synchronization action for user_article in my.conf.
   
   ```java
   public class UserArticleDatabasePreciseShardingAlgorithm implements PreciseShardingAlgorithm<Long> {
   
       /**
        * Sharding.
        *
        * @param availableTargetNames available data sources or tables's names
        * @param shardingValue        sharding value
        * @return sharding result for data source or table's name
        */
       @Override
       public String doSharding(Collection<String> availableTargetNames, PreciseShardingValue<Long> shardingValue) {
           if (CollUtil.isEmpty(availableTargetNames)) {
               throw new RuntimeException("无可用的数据源");
           }
           // 目前配置了三个数据库,master,slave0,slave1
           // 当前这个分库策略是给user_article用的,分库字段是user_id,userId对可用数据库进行取模
           Long value = shardingValue.getValue();
           long index = value % availableTargetNames.size();
   
           int i = 0;
           for (String availableTargetName : availableTargetNames) {
               if (i == index) {
                   return availableTargetName;
               }
               i ++;
           }
           return null;
       }
   }
   ```
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] dongfangding commented on issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
dongfangding commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-698253468


   Sorry, I still have some questions that I need to disturb.
   I'm just a newbie who just got in touch with this one.
   In my demo, I only hoped to do read-write separation at the beginning of the period, and assumed that this is enough for my application, but later I only want to sub-database with that individual table, and the overall architecture I still think read-write separation is Can satisfy me, is this kind of unreasonable existence?
   
   Isn’t this exactly what you stated in the documentation?
   
   https://shardingsphere.apache.org/document/legacy/4.x/document/cn/features/read-write-split/core-features/
   
   1. 提供一主多从的读写分离配置,可独立使用,也可配合分库分表使用。
   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero edited a comment on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-690858993






----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-696534740


   Hi @dongfangding 
   
   Sorry for late reply.  As I said before, the masterSlave dataSource is an important subject for ShardingSphere. We will give it a deep consideration and possible refactoring work before 5.x release. Currently, @terrymanu is taking charge of this issue. Any updates will be linked here, then you can follow up on this issue for its progress.


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] terrymanu closed issue #7269: dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
terrymanu closed issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269


   


----------------------------------------------------------------
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.

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



[GitHub] [shardingsphere] tristaZero commented on issue #7269: The question about dataSourceNames only taking the last one after master-slave-rules is configured

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #7269:
URL: https://github.com/apache/shardingsphere/issues/7269#issuecomment-689269800


   > I still have to reiterate my scenario, that is, multiple tables, some tables I want to use master-slave-rule, and other tables, I just want it to implement its own TableRule.
   
   Got it, could you rename master-slave rule to another one? Let's differentiate the name of master-slave Rule from the name of the real master data node.


----------------------------------------------------------------
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.

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