未验证 提交 dfac26ec 编写于 作者: Z zhyee 提交者: GitHub

Add support for shadow in proxy (#4689)

* add shadow-core-common  module;
add support for shadow to the PreparedStatementExecutorWrapper;
add support for shadow to the StatementExecutorWrapper

* add dependency in sharding-core-entry

* fix BootStrap and ConfigurationServive

* remove shadow-core-common

* add judge is shadow rule in ConfigurationService

* reset yaml

* reset server.yml
上级 79293b0c
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.orchestration.internal.registry.config.event;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.api.config.shadow.ShadowRuleConfiguration;
import org.apache.shardingsphere.orchestration.internal.registry.listener.ShardingOrchestrationEvent;
/**
* Shadow rule changed event.
*/
@RequiredArgsConstructor
@Getter
public class ShadowRuleChangedEvent implements ShardingOrchestrationEvent {
private final String shardingSchemaName;
private final ShadowRuleConfiguration shadowRuleConfiguration;
}
......@@ -23,16 +23,19 @@ import com.google.common.base.Splitter;
import com.google.common.base.Strings;
import org.apache.shardingsphere.api.config.masterslave.MasterSlaveRuleConfiguration;
import org.apache.shardingsphere.api.config.shadow.ShadowRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.ShardingRuleConfiguration;
import org.apache.shardingsphere.core.rule.Authentication;
import org.apache.shardingsphere.core.yaml.config.common.YamlAuthenticationConfiguration;
import org.apache.shardingsphere.core.yaml.config.masterslave.YamlMasterSlaveRuleConfiguration;
import org.apache.shardingsphere.core.yaml.config.shadow.YamlShadowRuleConfiguration;
import org.apache.shardingsphere.core.yaml.config.sharding.YamlShardingRuleConfiguration;
import org.apache.shardingsphere.core.yaml.constructor.YamlRootShardingConfigurationConstructor;
import org.apache.shardingsphere.core.yaml.representer.processor.ShardingTupleProcessorFactory;
import org.apache.shardingsphere.core.yaml.swapper.AuthenticationYamlSwapper;
import org.apache.shardingsphere.core.yaml.swapper.MasterSlaveRuleConfigurationYamlSwapper;
import org.apache.shardingsphere.core.yaml.swapper.ShardingRuleConfigurationYamlSwapper;
import org.apache.shardingsphere.core.yaml.swapper.impl.ShadowRuleConfigurationYamlSwapper;
import org.apache.shardingsphere.encrypt.api.EncryptRuleConfiguration;
import org.apache.shardingsphere.encrypt.yaml.config.YamlEncryptRuleConfiguration;
import org.apache.shardingsphere.encrypt.yaml.swapper.EncryptRuleConfigurationYamlSwapper;
......@@ -109,6 +112,8 @@ public final class ConfigurationService {
persistShardingRuleConfiguration(shardingSchemaName, (ShardingRuleConfiguration) ruleConfig);
} else if (ruleConfig instanceof EncryptRuleConfiguration) {
persistEncryptRuleConfiguration(shardingSchemaName, (EncryptRuleConfiguration) ruleConfig);
} else if (ruleConfig instanceof ShadowRuleConfiguration) {
persisShadowRuleConfiguration(shardingSchemaName, (ShadowRuleConfiguration) ruleConfig);
} else {
persistMasterSlaveRuleConfiguration(shardingSchemaName, (MasterSlaveRuleConfiguration) ruleConfig);
}
......@@ -139,6 +144,12 @@ public final class ConfigurationService {
configCenterRepository.persist(configNode.getRulePath(shardingSchemaName), YamlEngine.marshal(new EncryptRuleConfigurationYamlSwapper().swap(encryptRuleConfiguration)));
}
private void persisShadowRuleConfiguration(final String shardingSchemaName, final ShadowRuleConfiguration shadowRuleConfiguration) {
Preconditions.checkState(null != shadowRuleConfiguration && !shadowRuleConfiguration.getColumn().isEmpty() && null != shadowRuleConfiguration.getShadowMappings(),
"No available shadow rule configuration in `%s` for orchestration.", shardingSchemaName);
configCenterRepository.persist(configNode.getRulePath(shardingSchemaName), YamlEngine.marshal(new ShadowRuleConfigurationYamlSwapper().swap(shadowRuleConfiguration)));
}
private void persistMasterSlaveRuleConfiguration(final String shardingSchemaName, final MasterSlaveRuleConfiguration masterSlaveRuleConfiguration) {
Preconditions.checkState(null != masterSlaveRuleConfiguration && !masterSlaveRuleConfiguration.getMasterDataSourceName().isEmpty(),
"No available master-slave rule configuration in `%s` for orchestration.", shardingSchemaName);
......@@ -210,6 +221,16 @@ public final class ConfigurationService {
&& configCenterRepository.get(configNode.getRulePath(shardingSchemaName)).contains("encryptors:\n");
}
/**
* Judge is shadow rule or not.
* @param shardingSchemaName sharding schema name
* @return is shadow rule or not
*/
public boolean isShadowRule(final String shardingSchemaName) {
return !configCenterRepository.get(configNode.getRulePath(shardingSchemaName)).contains("shadowRule:\n")
&& configCenterRepository.get(configNode.getRulePath(shardingSchemaName)).contains("shadowMappings:\n");
}
/**
* Load data source configurations.
*
......@@ -254,6 +275,16 @@ public final class ConfigurationService {
return new EncryptRuleConfigurationYamlSwapper().swap(YamlEngine.unmarshal(configCenterRepository.get(configNode.getRulePath(shardingSchemaName)), YamlEncryptRuleConfiguration.class));
}
/**
* Load shadow rule configuration.
*
* @param shardingSchemaName sharding schema name
* @return shadow rule configuration
*/
public ShadowRuleConfiguration loadShadowRuleConfiguration(final String shardingSchemaName) {
return new ShadowRuleConfigurationYamlSwapper().swap(YamlEngine.unmarshal(configCenterRepository.get(configNode.getRulePath(shardingSchemaName)), YamlShadowRuleConfiguration.class));
}
/**
* Load authentication.
*
......
......@@ -19,15 +19,18 @@ package org.apache.shardingsphere.orchestration.internal.registry.config.service
import org.apache.commons.dbcp2.BasicDataSource;
import org.apache.shardingsphere.api.config.masterslave.MasterSlaveRuleConfiguration;
import org.apache.shardingsphere.api.config.shadow.ShadowRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.ShardingRuleConfiguration;
import org.apache.shardingsphere.core.rule.Authentication;
import org.apache.shardingsphere.core.yaml.config.common.YamlAuthenticationConfiguration;
import org.apache.shardingsphere.core.yaml.config.masterslave.YamlMasterSlaveRuleConfiguration;
import org.apache.shardingsphere.core.yaml.config.shadow.YamlShadowRuleConfiguration;
import org.apache.shardingsphere.core.yaml.config.sharding.YamlShardingRuleConfiguration;
import org.apache.shardingsphere.core.yaml.constructor.YamlRootShardingConfigurationConstructor;
import org.apache.shardingsphere.core.yaml.swapper.AuthenticationYamlSwapper;
import org.apache.shardingsphere.core.yaml.swapper.MasterSlaveRuleConfigurationYamlSwapper;
import org.apache.shardingsphere.core.yaml.swapper.ShardingRuleConfigurationYamlSwapper;
import org.apache.shardingsphere.core.yaml.swapper.impl.ShadowRuleConfigurationYamlSwapper;
import org.apache.shardingsphere.encrypt.api.EncryptRuleConfiguration;
import org.apache.shardingsphere.encrypt.api.EncryptorRuleConfiguration;
import org.apache.shardingsphere.encrypt.yaml.config.YamlEncryptRuleConfiguration;
......@@ -88,6 +91,10 @@ public final class ConfigurationServiceTest {
+ " order_id:\n"
+ " cipherColumn: order_id\n" + " encryptor: order_encryptor\n";
private static final String SHADOW_RULE_YAML = "column: shadow\n"
+ "shadowMappings:\n"
+ " ds: shadow_ds\n";
private static final String AUTHENTICATION_YAML = "users:\n" + " root1:\n" + " authorizedSchemas: sharding_db\n" + " password: root1\n"
+ " root2:\n" + " authorizedSchemas: sharding_db,ms_db\n" + " password: root2\n";
......@@ -247,6 +254,14 @@ public final class ConfigurationServiceTest {
verify(regCenter).persist("/test/config/schema/sharding_db/rule", ENCRYPT_RULE_YAML);
}
@Test
public void assertPersistConfigurationForShadow() {
ConfigurationService configurationService = new ConfigurationService("test", regCenter);
configurationService.persistConfiguration("sharding_db", createDataSourceConfigurations(), createShadowRuleConfiguration(), null, createProperties(), true);
verify(regCenter).persist(eq("/test/config/schema/sharding_db/datasource"), ArgumentMatchers.any());
verify(regCenter).persist("/test/config/schema/sharding_db/rule", SHADOW_RULE_YAML);
}
private Map<String, DataSourceConfiguration> createDataSourceConfigurations() {
return createDataSourceMap().entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> DataSourceConfiguration.getDataSourceConfiguration(e.getValue())));
}
......@@ -283,6 +298,10 @@ public final class ConfigurationServiceTest {
return new EncryptRuleConfigurationYamlSwapper().swap(YamlEngine.unmarshal(ENCRYPT_RULE_YAML, YamlEncryptRuleConfiguration.class));
}
private ShadowRuleConfiguration createShadowRuleConfiguration() {
return new ShadowRuleConfigurationYamlSwapper().swap(YamlEngine.unmarshal(SHADOW_RULE_YAML, YamlShadowRuleConfiguration.class));
}
private Authentication createAuthentication() {
return new AuthenticationYamlSwapper().swap(YamlEngine.unmarshal(AUTHENTICATION_YAML, YamlAuthenticationConfiguration.class));
}
......@@ -331,6 +350,13 @@ public final class ConfigurationServiceTest {
assertTrue(configurationService.isEncryptRule("sharding_db"));
}
@Test
public void assertIsShadowRule() {
when(regCenter.get("/test/config/schema/sharding_db/rule")).thenReturn(SHADOW_RULE_YAML);
ConfigurationService configurationService = new ConfigurationService("test", regCenter);
assertTrue(configurationService.isShadowRule("sharding_db"));
}
@Test
public void assertIsNotShardingRule() {
when(regCenter.get("/test/config/schema/sharding_db/rule")).thenReturn(MASTER_SLAVE_RULE_YAML);
......@@ -367,6 +393,15 @@ public final class ConfigurationServiceTest {
assertThat(entry.getValue().getProperties().get("aes.key.value").toString(), is("123456"));
}
@Test
public void assertLoadShadowRuleConfiguration() {
when(regCenter.get("/test/config/schema/sharding_db/rule")).thenReturn(SHADOW_RULE_YAML);
ConfigurationService configurationService = new ConfigurationService("test", regCenter);
ShadowRuleConfiguration actual = configurationService.loadShadowRuleConfiguration("sharding_db");
assertThat(actual.getShadowMappings().get("ds"), is("shadow_ds"));
assertThat(actual.getColumn(), is("shadow"));
}
@Test
public void assertLoadAuthentication() {
when(regCenter.get("/test/config/authentication")).thenReturn(AUTHENTICATION_YAML);
......
......@@ -18,7 +18,12 @@
package org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.wrapper;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.rule.ShadowRule;
import org.apache.shardingsphere.masterslave.route.engine.MasterSlaveRouter;
import org.apache.shardingsphere.shadow.rewrite.context.ShadowSQLRewriteContextDecorator;
import org.apache.shardingsphere.shadow.rewrite.judgement.ShadowJudgementEngine;
import org.apache.shardingsphere.shadow.rewrite.judgement.impl.PreparedJudgementEngine;
import org.apache.shardingsphere.shardingproxy.backend.schema.impl.ShadowSchema;
import org.apache.shardingsphere.underlying.route.context.RouteUnit;
import org.apache.shardingsphere.core.shard.PreparedQueryShardingEngine;
import org.apache.shardingsphere.underlying.executor.context.ExecutionContext;
......@@ -76,6 +81,9 @@ public final class PreparedStatementExecutorWrapper implements JDBCExecutorWrapp
if (logicSchema instanceof EncryptSchema) {
return doEncryptRoute(sql);
}
if (logicSchema instanceof ShadowSchema) {
return doShadowRoute(sql);
}
return doTransparentRoute(sql);
}
......@@ -114,9 +122,31 @@ public final class PreparedStatementExecutorWrapper implements JDBCExecutorWrapp
return result;
}
private Map<BaseRule, SQLRewriteContextDecorator> createSQLRewriteContextDecorator(final EncryptRule encryptRule) {
private ExecutionContext doShadowRoute(final String sql) {
ShadowSchema shadowSchema = (ShadowSchema) logicSchema;
SQLStatement sqlStatement = shadowSchema.getSqlParserEngine().parse(sql, true);
RelationMetas relationMetas = logicSchema.getMetaData().getRelationMetas();
SQLStatementContext sqlStatementContext = SQLStatementContextFactory.newInstance(relationMetas, sql, parameters, sqlStatement);
ShadowJudgementEngine shadowJudgementEngine = new PreparedJudgementEngine(shadowSchema.getShadowRule(), sqlStatementContext, parameters);
SQLRewriteContext sqlRewriteContext = new SQLRewriteEntry(logicSchema.getMetaData(), ShardingProxyContext.getInstance().getProperties())
.createSQLRewriteContext(sql, parameters, sqlStatementContext, createSQLRewriteContextDecorator(shadowSchema.getShadowRule()));
SQLRewriteResult sqlRewriteResult = new DefaultSQLRewriteEngine().rewrite(sqlRewriteContext);
ExecutionContext result = new ExecutionContext(sqlStatementContext);
String dataSourceName = shadowJudgementEngine.isShadowSQL()
? shadowSchema.getShadowRule().getRuleConfiguration().getShadowMappings().get(logicSchema.getDataSources().keySet().iterator().next())
: logicSchema.getDataSources().keySet().iterator().next();
result.getExecutionUnits().add(
new ExecutionUnit(dataSourceName, new SQLUnit(sqlRewriteResult.getSql(), sqlRewriteResult.getParameters())));
return result;
}
private Map<BaseRule, SQLRewriteContextDecorator> createSQLRewriteContextDecorator(final BaseRule baseRule) {
Map<BaseRule, SQLRewriteContextDecorator> result = new HashMap<>(1, 1);
result.put(encryptRule, new EncryptSQLRewriteContextDecorator());
if (baseRule instanceof EncryptRule) {
result.put(baseRule, new EncryptSQLRewriteContextDecorator());
} else if (baseRule instanceof ShadowRule) {
result.put(baseRule, new ShadowSQLRewriteContextDecorator());
}
return result;
}
......
......@@ -18,7 +18,12 @@
package org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.wrapper;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.rule.ShadowRule;
import org.apache.shardingsphere.masterslave.route.engine.MasterSlaveRouter;
import org.apache.shardingsphere.shadow.rewrite.context.ShadowSQLRewriteContextDecorator;
import org.apache.shardingsphere.shadow.rewrite.judgement.ShadowJudgementEngine;
import org.apache.shardingsphere.shadow.rewrite.judgement.impl.SimpleJudgementEngine;
import org.apache.shardingsphere.shardingproxy.backend.schema.impl.ShadowSchema;
import org.apache.shardingsphere.underlying.route.context.RouteUnit;
import org.apache.shardingsphere.core.shard.SimpleQueryShardingEngine;
import org.apache.shardingsphere.underlying.executor.context.ExecutionContext;
......@@ -73,6 +78,9 @@ public final class StatementExecutorWrapper implements JDBCExecutorWrapper {
if (logicSchema instanceof EncryptSchema) {
return doEncryptRoute(sql);
}
if (logicSchema instanceof ShadowSchema) {
return doShadowRoute(sql);
}
return doTransparentRoute(sql);
}
......@@ -111,9 +119,31 @@ public final class StatementExecutorWrapper implements JDBCExecutorWrapper {
return result;
}
private Map<BaseRule, SQLRewriteContextDecorator> createSQLRewriteContextDecorator(final EncryptRule encryptRule) {
private ExecutionContext doShadowRoute(final String sql) {
ShadowSchema shadowSchema = (ShadowSchema) logicSchema;
SQLStatement sqlStatement = shadowSchema.getSqlParserEngine().parse(sql, true);
RelationMetas relationMetas = logicSchema.getMetaData().getRelationMetas();
SQLStatementContext sqlStatementContext = SQLStatementContextFactory.newInstance(relationMetas, sql, new LinkedList<>(), sqlStatement);
ShadowJudgementEngine shadowJudgementEngine = new SimpleJudgementEngine(shadowSchema.getShadowRule(), sqlStatementContext);
String dataSourceName = shadowJudgementEngine.isShadowSQL()
? shadowSchema.getShadowRule().getRuleConfiguration().getShadowMappings().get(logicSchema.getDataSources().keySet().iterator().next())
: logicSchema.getDataSources().keySet().iterator().next();
SQLRewriteContext sqlRewriteContext = new SQLRewriteEntry(logicSchema.getMetaData(), ShardingProxyContext.getInstance().getProperties())
.createSQLRewriteContext(sql, Collections.emptyList(), sqlStatementContext, createSQLRewriteContextDecorator(shadowSchema.getShadowRule()));
SQLRewriteResult sqlRewriteResult = new DefaultSQLRewriteEngine().rewrite(sqlRewriteContext);
ExecutionContext result = new ExecutionContext(sqlStatementContext);
result.getExecutionUnits().add(
new ExecutionUnit(dataSourceName, new SQLUnit(sqlRewriteResult.getSql(), sqlRewriteResult.getParameters())));
return result;
}
private Map<BaseRule, SQLRewriteContextDecorator> createSQLRewriteContextDecorator(final BaseRule baseRule) {
Map<BaseRule, SQLRewriteContextDecorator> result = new HashMap<>(1, 1);
result.put(encryptRule, new EncryptSQLRewriteContextDecorator());
if (baseRule instanceof EncryptRule) {
result.put(baseRule, new EncryptSQLRewriteContextDecorator());
} else if (baseRule instanceof ShadowRule) {
result.put(baseRule, new ShadowSQLRewriteContextDecorator());
}
return result;
}
......
......@@ -19,6 +19,8 @@ package org.apache.shardingsphere.shardingproxy.backend.schema;
import lombok.AccessLevel;
import lombok.NoArgsConstructor;
import org.apache.shardingsphere.api.config.shadow.ShadowRuleConfiguration;
import org.apache.shardingsphere.shardingproxy.backend.schema.impl.ShadowSchema;
import org.apache.shardingsphere.underlying.common.config.RuleConfiguration;
import org.apache.shardingsphere.encrypt.api.EncryptRuleConfiguration;
import org.apache.shardingsphere.api.config.masterslave.MasterSlaveRuleConfiguration;
......@@ -59,6 +61,9 @@ public final class LogicSchemaFactory {
if (ruleConfiguration instanceof EncryptRuleConfiguration) {
return new EncryptSchema(schemaName, schemaDataSources.get(schemaName), (EncryptRuleConfiguration) ruleConfiguration);
}
if (ruleConfiguration instanceof ShadowRuleConfiguration) {
return new ShadowSchema(schemaName, schemaDataSources.get(schemaName), (ShadowRuleConfiguration) ruleConfiguration);
}
return new TransparentSchema(schemaName, schemaDataSources.get(schemaName));
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.shardingsphere.shardingproxy.backend.schema.impl;
import com.google.common.eventbus.Subscribe;
import lombok.Getter;
import lombok.SneakyThrows;
import org.apache.shardingsphere.api.config.shadow.ShadowRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.ShardingRuleConfiguration;
import org.apache.shardingsphere.core.log.ConfigurationLogger;
import org.apache.shardingsphere.core.rule.ShadowRule;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.orchestration.internal.registry.config.event.ShadowRuleChangedEvent;
import org.apache.shardingsphere.shardingproxy.backend.schema.LogicSchema;
import org.apache.shardingsphere.shardingproxy.backend.schema.LogicSchemas;
import org.apache.shardingsphere.shardingproxy.backend.schema.ProxyConnectionManager;
import org.apache.shardingsphere.shardingproxy.config.yaml.YamlDataSourceParameter;
import org.apache.shardingsphere.underlying.common.metadata.ShardingSphereMetaData;
import org.apache.shardingsphere.underlying.common.metadata.datasource.DataSourceMetas;
import org.apache.shardingsphere.underlying.common.metadata.table.TableMetas;
import org.apache.shardingsphere.underlying.common.metadata.table.init.TableMetaDataInitializer;
import org.apache.shardingsphere.underlying.common.metadata.table.init.TableMetaDataInitializerEntry;
import org.apache.shardingsphere.underlying.common.metadata.table.init.loader.impl.DefaultTableMetaDataLoader;
import org.apache.shardingsphere.underlying.common.rule.BaseRule;
import java.sql.SQLException;
import java.util.HashMap;
import java.util.Map;
/**
* Shadow schema.
*/
@Getter
public final class ShadowSchema extends LogicSchema {
private final ShardingSphereMetaData metaData;
private final ShardingRule shardingRule;
private ShadowRule shadowRule;
public ShadowSchema(final String name, final Map<String, YamlDataSourceParameter> dataSources, final ShadowRuleConfiguration shadowRuleConfiguration) throws SQLException {
super(name, dataSources);
shadowRule = new ShadowRule(shadowRuleConfiguration);
shardingRule = new ShardingRule(new ShardingRuleConfiguration(), getDataSources().keySet());
metaData = createMetaData();
}
private ShardingSphereMetaData createMetaData() throws SQLException {
DataSourceMetas dataSourceMetas = new DataSourceMetas(LogicSchemas.getInstance().getDatabaseType(), getDatabaseAccessConfigurationMap());
TableMetas tableMetas = createTableMetaDataInitializerEntry(dataSourceMetas).initAll();
return new ShardingSphereMetaData(dataSourceMetas, tableMetas);
}
private TableMetaDataInitializerEntry createTableMetaDataInitializerEntry(final DataSourceMetas dataSourceMetas) {
Map<BaseRule, TableMetaDataInitializer> tableMetaDataInitializes = new HashMap<>(1, 1);
tableMetaDataInitializes.put(shadowRule, new DefaultTableMetaDataLoader(dataSourceMetas, new ProxyConnectionManager(getBackendDataSource())));
return new TableMetaDataInitializerEntry(tableMetaDataInitializes);
}
/**
* Renew shadow rule.
*
* @param shadowRuleChangedEvent shadow configuration changed event
*/
@Subscribe
@SneakyThrows
public synchronized void renew(final ShadowRuleChangedEvent shadowRuleChangedEvent) {
ConfigurationLogger.log(shadowRuleChangedEvent.getShadowRuleConfiguration());
shadowRule = new ShadowRule(shadowRuleChangedEvent.getShadowRuleConfiguration());
}
}
......@@ -21,6 +21,7 @@ import com.google.common.primitives.Ints;
import lombok.AccessLevel;
import lombok.NoArgsConstructor;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.shardingsphere.core.yaml.swapper.impl.ShadowRuleConfigurationYamlSwapper;
import org.apache.shardingsphere.underlying.common.config.RuleConfiguration;
import org.apache.shardingsphere.underlying.common.config.DataSourceConfiguration;
import org.apache.shardingsphere.underlying.common.constant.properties.PropertiesConstant;
......@@ -146,6 +147,8 @@ public final class Bootstrap {
result.put(each, shardingOrchestrationFacade.getConfigService().loadEncryptRuleConfiguration(each));
} else if (shardingOrchestrationFacade.getConfigService().isShardingRule(each)) {
result.put(each, shardingOrchestrationFacade.getConfigService().loadShardingRuleConfiguration(each));
} else if (shardingOrchestrationFacade.getConfigService().isShadowRule(each)) {
result.put(each, shardingOrchestrationFacade.getConfigService().loadShadowRuleConfiguration(each));
} else {
result.put(each, shardingOrchestrationFacade.getConfigService().loadMasterSlaveRuleConfiguration(each));
}
......@@ -194,6 +197,8 @@ public final class Bootstrap {
result.put(entry.getKey(), new MasterSlaveRuleConfigurationYamlSwapper().swap(entry.getValue().getMasterSlaveRule()));
} else if (null != entry.getValue().getEncryptRule()) {
result.put(entry.getKey(), new EncryptRuleConfigurationYamlSwapper().swap(entry.getValue().getEncryptRule()));
} else if (null != entry.getValue().getShadowRule()) {
result.put(entry.getKey(), new ShadowRuleConfigurationYamlSwapper().swap(entry.getValue().getShadowRule()));
}
}
return result;
......
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
######################################################################################################
#
# Here you can configure the rules for the proxy.
# This example is configuration of sharding rule.
#
# If you want to use sharding, please refer to this file;
# if you want to use master-slave, please refer to the config-master_slave.yaml.
# if you want to use master-slave, please refer to the config-encrypt.yaml.
#
######################################################################################################
#
#schemaName: sharding_db
#
#dataSources:
# ds:
# url: jdbc:postgresql://127.0.0.1:5432/demo_ds_0?serverTimezone=UTC&useSSL=false
# username: postgres
# password: postgres
# connectionTimeoutMilliseconds: 30000
# idleTimeoutMilliseconds: 60000
# maxLifetimeMilliseconds: 1800000
# maxPoolSize: 50
# shadow_ds:
# url: jdbc:postgresql://127.0.0.1:5432/demo_ds_1?serverTimezone=UTC&useSSL=false
# username: postgres
# password: postgres
# connectionTimeoutMilliseconds: 30000
# idleTimeoutMilliseconds: 60000
# maxLifetimeMilliseconds: 1800000
# maxPoolSize: 50
#
#shadowRule:
# column: shadow
# shadowMappings:
# ds: shadow_ds
#
#
######################################################################################################
#
# If you want to connect to MySQL, you should manually copy MySQL driver to lib directory.
#
######################################################################################################
#
#schemaName: sharding_db
#
#dataSources:
# ds:
# url: jdbc:mysql://127.0.0.1:3306/demo_ds_0?serverTimezone=UTC&useSSL=false
# username: root
# password:
# connectionTimeoutMilliseconds: 30000
# idleTimeoutMilliseconds: 60000
# maxLifetimeMilliseconds: 1800000
# maxPoolSize: 50
# shadow_ds:
# url: jdbc:mysql://127.0.0.1:3306/demo_ds_1?serverTimezone=UTC&useSSL=false
# username: root
# password:
# connectionTimeoutMilliseconds: 30000
# idleTimeoutMilliseconds: 60000
# maxLifetimeMilliseconds: 1800000
# maxPoolSize: 50
#
#shadowRule:
# column: shadow
# shadowMappings:
# ds: shadow_ds
......@@ -19,6 +19,7 @@ package org.apache.shardingsphere.shardingproxy.config.yaml;
import lombok.Getter;
import lombok.Setter;
import org.apache.shardingsphere.core.yaml.config.shadow.YamlShadowRuleConfiguration;
import org.apache.shardingsphere.underlying.common.yaml.config.YamlConfiguration;
import org.apache.shardingsphere.encrypt.yaml.config.YamlEncryptRuleConfiguration;
import org.apache.shardingsphere.core.yaml.config.masterslave.YamlMasterSlaveRuleConfiguration;
......@@ -45,4 +46,6 @@ public final class YamlProxyRuleConfiguration implements YamlConfiguration {
private YamlMasterSlaveRuleConfiguration masterSlaveRule;
private YamlEncryptRuleConfiguration encryptRule;
private YamlShadowRuleConfiguration shadowRule;
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册