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 2022/11/13 09:20:09 UTC

[GitHub] [shardingsphere] jianzh5 opened a new issue, #22140: how to config actual-data-nodes while my actual-data-nodes tables is dynamic

jianzh5 opened a new issue, #22140:
URL: https://github.com/apache/shardingsphere/issues/22140

   hello,everyone:
   
   i want to config actual-data-nodes while my actual-data-nodes tables is dynamic ,i found  solution  through this issue https://github.com/apache/shardingsphere/issues/16725。
   
   when i try it on sharding-jdbc 5.2.1 , this method  can change actual-data-nodes actually,but lose KeyGenerators,This error is reported at startup。
   
   ```java
   ### Cause: java.sql.SQLException: Field 'EVENT_ID' doesn't have a default value
   ; Field 'EVENT_ID' doesn't have a default value; nested exception is java.sql.SQLException: Field 'EVENT_ID' doesn't have a default value
   ```
   
   ```java
   public class InitActualDataNodesExample implements ApplicationRunner {
       private final DataSource dataSource;
   
       @Override
       public void run(ApplicationArguments args)  {
           initActualDataNodes((ShardingSphereDataSource) dataSource);
       }
   
       private void initActualDataNodes(ShardingSphereDataSource dataSource) {
   
   
           // 更新 context
           ContextManager contextManager = getContextManager(dataSource);
           Collection<RuleConfiguration> addRuleConfigs = new LinkedList<>();
           Collection<RuleConfiguration> configurations = contextManager.getMetaDataContexts().getMetaData().getDatabases()
                   .get("logic_db").getRuleMetaData().getConfigurations();
   
           for (RuleConfiguration configuration : configurations) {
               if(configuration instanceof ShardingRuleConfiguration){
                   ShardingRuleConfiguration algorithmProvidedShardingRuleConfiguration = (ShardingRuleConfiguration) configuration;
   
   
                   ShardingRuleConfiguration addRuleConfiguration = new ShardingRuleConfiguration();
                   Collection<ShardingTableRuleConfiguration> addTableConfigurations = new LinkedList<>();
                   for (ShardingTableRuleConfiguration shardingTableRuleConfiguration : algorithmProvidedShardingRuleConfiguration.getTables()) {
   
                       if ("T_EVENT".equals(shardingTableRuleConfiguration.getLogicTable())) {
                           String actualDataNodes2 = "ds$->{0..1}.T_EVENT_$->{20221111..20221121}";
                           ShardingTableRuleConfiguration addTableConfiguration = new ShardingTableRuleConfiguration(shardingTableRuleConfiguration.getLogicTable(), actualDataNodes2);
                           addTableConfiguration.setTableShardingStrategy(shardingTableRuleConfiguration.getTableShardingStrategy());
                           addTableConfiguration.setDatabaseShardingStrategy(shardingTableRuleConfiguration.getDatabaseShardingStrategy());
                           addTableConfiguration.setKeyGenerateStrategy(shardingTableRuleConfiguration.getKeyGenerateStrategy());
                           addTableConfigurations.add(addTableConfiguration);
                       } else {
                           addTableConfigurations.add(shardingTableRuleConfiguration);
                       }
                   }
   
                   addRuleConfiguration.setTables(addTableConfigurations);
                   addRuleConfiguration.setAutoTables(algorithmProvidedShardingRuleConfiguration.getAutoTables());
                   addRuleConfiguration.setBindingTableGroups(algorithmProvidedShardingRuleConfiguration.getBindingTableGroups());
                   addRuleConfiguration.setBroadcastTables(algorithmProvidedShardingRuleConfiguration.getBroadcastTables());
                   addRuleConfiguration.setDefaultDatabaseShardingStrategy(algorithmProvidedShardingRuleConfiguration.getDefaultDatabaseShardingStrategy());
                   addRuleConfiguration.setDefaultTableShardingStrategy(algorithmProvidedShardingRuleConfiguration.getDefaultTableShardingStrategy());
                   addRuleConfiguration.setDefaultKeyGenerateStrategy(algorithmProvidedShardingRuleConfiguration.getDefaultKeyGenerateStrategy());
                   addRuleConfiguration.setDefaultShardingColumn(algorithmProvidedShardingRuleConfiguration.getDefaultShardingColumn());
                   addRuleConfiguration.setShardingAlgorithms(algorithmProvidedShardingRuleConfiguration.getShardingAlgorithms());
                   addRuleConfiguration.setKeyGenerators(algorithmProvidedShardingRuleConfiguration.getKeyGenerators());
                   addRuleConfigs.add(addRuleConfiguration);
               }else{
                   addRuleConfigs.add(configuration);
               }
   
           }
           contextManager.alterRuleConfiguration("logic_db", addRuleConfigs);
       }
   
       private ContextManager getContextManager(ShardingSphereDataSource dataSource) {
           try {
               Field contextManagerField = dataSource.getClass().getDeclaredField("contextManager");
               contextManagerField.setAccessible(true);
               return (ContextManager) contextManagerField.get(dataSource);
           }
           catch (Exception e) {
               throw new RuntimeException("系统异常");
           }
   
       }
   }
   ```
   
   


-- 
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: notifications-unsubscribe@shardingsphere.apache.org.apache.org

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


[GitHub] [shardingsphere] EvanJonQiu commented on issue #22140: how to config actual-data-nodes while my actual-data-nodes tables is dynamic

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

   I have the same problem.
   
   I change ctual-data-nodes just like @jianzh5's code. When I insert data, the key generator not works.
   
   It should be:
       {
           "id": 818611524267409408,  <------ without change sharding rule
           "orderId": 1,
           "orderMessage": "1",
           "createTime": "2022-01-01 02:10:10"
       },
   but it was:
       {
           "id": 5,       <------- key generator not work
           "orderId": 1,
           "orderMessage": "1",
           "createTime": "2022-01-01 02:10:10"
       }
   
   My code:
   `
   @Override
   	public void run(ApplicationArguments args) throws Exception {
   
   		logger.debug("in " + this.getClass().getName() + "::run()");
   
   		//ContextManager contextManager = this.getContextManager(this.shardingSphereDataSource);
   		ContextManager contextManager = this.getContextManager((ShardingSphereDataSource)dataSource);
   		
   		Collection<RuleConfiguration> newRuleConfigurations = new LinkedList<>();
   		
   		Collection<RuleConfiguration> oldRuleConfigurations = contextManager.getMetaDataContexts().getMetaData().getDatabases()
   				.get("logic_db").getRuleMetaData().getConfigurations();
   		
   		ShardingSphereDatabase database = contextManager.getMetaDataContexts().getMetaData().getDatabases().get("logic_db");
   		
   		for (RuleConfiguration config: oldRuleConfigurations) {
   			if (config instanceof ShardingRuleConfiguration) {
   				
   				ShardingRuleConfiguration oldShardingRuleConfig = (ShardingRuleConfiguration)config;
   				
   				ShardingRuleConfiguration newShardingRuleConfig = new ShardingRuleConfiguration();
   				// 根据分片算法名称获取分片算法配置
   				Map<String, AlgorithmConfiguration> oldAlgorithmConfigs = oldShardingRuleConfig.getShardingAlgorithms();
   				AlgorithmConfiguration algorithmConfiguration = oldAlgorithmConfigs.get(algorithm_name);
   				
   				Collection<ShardingTableRuleConfiguration> oldShardingTableRuleConfigs = oldShardingRuleConfig.getTables();
   				Collection<ShardingTableRuleConfiguration> newShardingTableRuleConfigs = new LinkedList<>();
   				
   				oldShardingTableRuleConfigs.forEach(shardingTableRuleConfigItem -> {
   					if (logic_table_name.equals(shardingTableRuleConfigItem.getLogicTable())) {
   						
   						String algorithmClassName = algorithmConfiguration.getProps().getProperty("algorithmClassName");
   						String actualDataNodes = "";
   						
   						try {
   							
   							actualDataNodes = "ds0.t_auto_create_table_${20221115..20221118}";
   							
   							ShardingTableRuleConfiguration newShardingTableRuleConfiguration = new ShardingTableRuleConfiguration(shardingTableRuleConfigItem.getLogicTable(), actualDataNodes);
   							newShardingTableRuleConfiguration.setTableShardingStrategy(shardingTableRuleConfigItem.getTableShardingStrategy());							
   							newShardingTableRuleConfiguration.setDatabaseShardingStrategy(shardingTableRuleConfigItem.getDatabaseShardingStrategy());
   							
   							KeyGenerateStrategyConfiguration oldKeyGenerateStrategyConfiguration = shardingTableRuleConfigItem.getKeyGenerateStrategy();
   							
   							newShardingTableRuleConfiguration.setKeyGenerateStrategy(shardingTableRuleConfigItem.getKeyGenerateStrategy());
   							
   							newShardingTableRuleConfigs.add(newShardingTableRuleConfiguration);
   						} catch (Exception e) {
   							throw new RuntimeException();
   						}
   					} else {
   						newShardingTableRuleConfigs.add(shardingTableRuleConfigItem);
   					}
   				});
   				
   				newShardingRuleConfig.setTables(newShardingTableRuleConfigs);
   				newShardingRuleConfig.setAutoTables(oldShardingRuleConfig.getAutoTables());
   				newShardingRuleConfig.setBindingTableGroups(oldShardingRuleConfig.getBindingTableGroups());
   				newShardingRuleConfig.setBroadcastTables(oldShardingRuleConfig.getBroadcastTables());
   				
   				 Map<String, AlgorithmConfiguration> oldKeyGenerators = oldShardingRuleConfig.getKeyGenerators();
   				
   				newShardingRuleConfig.setKeyGenerators(oldShardingRuleConfig.getKeyGenerators());
   				newShardingRuleConfig.setShardingAlgorithms(oldShardingRuleConfig.getShardingAlgorithms());
   				
   				newShardingRuleConfig.setDefaultAuditStrategy(oldShardingRuleConfig.getDefaultAuditStrategy());
   				newShardingRuleConfig.setDefaultDatabaseShardingStrategy(oldShardingRuleConfig.getDefaultDatabaseShardingStrategy());
   				newShardingRuleConfig.setDefaultKeyGenerateStrategy(oldShardingRuleConfig.getDefaultKeyGenerateStrategy());
   				newShardingRuleConfig.setDefaultShardingColumn(oldShardingRuleConfig.getDefaultShardingColumn());
   				newShardingRuleConfig.setDefaultTableShardingStrategy(oldShardingRuleConfig.getDefaultTableShardingStrategy());
   				
   				newRuleConfigurations.add(newShardingRuleConfig);
   			} else {
   				newRuleConfigurations.add(config);
   			}
   		}
   		
   		contextManager.alterRuleConfiguration(DATABASE_NAME, newRuleConfigurations);
   		
   	}
   `


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] jianzh5 commented on issue #22140: how to config actual-data-nodes while my actual-data-nodes tables is dynamic

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

   ```
   key-generators:
             key-cosid_snowflake:
               type: COSID_SNOWFLAKE
   ```
   it din't works
   
   but if i use snowflake by mybatis-plus ,it works


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] TeslaCN commented on issue #22140: KeyGenerator COSID_SNOWFLAKE does not take effect

Posted by "TeslaCN (via GitHub)" <gi...@apache.org>.
TeslaCN commented on issue #22140:
URL: https://github.com/apache/shardingsphere/issues/22140#issuecomment-1480542836

   Close due to no response.


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] RaigorJiang commented on issue #22140: how to config actual-data-nodes while my actual-data-nodes tables is dynamic

Posted by "RaigorJiang (via GitHub)" <gi...@apache.org>.
RaigorJiang commented on issue #22140:
URL: https://github.com/apache/shardingsphere/issues/22140#issuecomment-1407544221

   @jianzh5 
   Please provide the original full sharding configuration so that the issue can be investigated.


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] azexcy commented on issue #22140: how to config actual-data-nodes while my actual-data-nodes tables is dynamic

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

   Hi @jianzh5, Maybe you can provide more information.


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] TeslaCN closed issue #22140: KeyGenerator COSID_SNOWFLAKE does not take effect

Posted by "TeslaCN (via GitHub)" <gi...@apache.org>.
TeslaCN closed issue #22140: KeyGenerator COSID_SNOWFLAKE does not take effect
URL: https://github.com/apache/shardingsphere/issues/22140


-- 
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: notifications-unsubscribe@shardingsphere.apache.org

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