提交 c4f7b476 编写于 作者: T terrymanu

Remove impl of GovernanceSchemaContexts

上级 ff9129fe
......@@ -68,13 +68,13 @@ import java.util.stream.Collectors;
/**
* Governance schema contexts.
*/
public class GovernanceSchemaContexts implements SchemaContexts {
public final class GovernanceSchemaContexts implements SchemaContexts {
private final GovernanceFacade governanceFacade;
private volatile SchemaContexts schemaContexts;
protected GovernanceSchemaContexts(final SchemaContexts schemaContexts, final GovernanceFacade governanceFacade) {
public GovernanceSchemaContexts(final SchemaContexts schemaContexts, final GovernanceFacade governanceFacade) {
this.governanceFacade = governanceFacade;
this.schemaContexts = schemaContexts;
ShardingSphereEventBus.getInstance().register(this);
......@@ -101,7 +101,7 @@ public class GovernanceSchemaContexts implements SchemaContexts {
}
@Override
public final DatabaseType getDatabaseType() {
public DatabaseType getDatabaseType() {
return schemaContexts.getDatabaseType();
}
......@@ -116,32 +116,32 @@ public class GovernanceSchemaContexts implements SchemaContexts {
}
@Override
public final Map<String, SchemaContext> getSchemaContexts() {
public Map<String, SchemaContext> getSchemaContexts() {
return schemaContexts.getSchemaContexts();
}
@Override
public final SchemaContext getDefaultSchemaContext() {
public SchemaContext getDefaultSchemaContext() {
return schemaContexts.getDefaultSchemaContext();
}
@Override
public final Authentication getAuthentication() {
public Authentication getAuthentication() {
return schemaContexts.getAuthentication();
}
@Override
public final ConfigurationProperties getProps() {
public ConfigurationProperties getProps() {
return schemaContexts.getProps();
}
@Override
public final boolean isCircuitBreak() {
public boolean isCircuitBreak() {
return schemaContexts.isCircuitBreak();
}
@Override
public final void close() throws Exception {
public void close() throws Exception {
schemaContexts.close();
governanceFacade.close();
}
......@@ -306,7 +306,7 @@ public class GovernanceSchemaContexts implements SchemaContexts {
return builder.build().getSchemaContexts().values().iterator().next();
}
private SchemaContext getChangedSchemaContext(final SchemaContext oldSchemaContext, final Map<String, DataSourceConfiguration> newDataSources) throws Exception {
private SchemaContext getChangedSchemaContext(final SchemaContext oldSchemaContext, final Map<String, DataSourceConfiguration> newDataSources) throws SQLException {
Collection<String> deletedDataSources = getDeletedDataSources(oldSchemaContext, newDataSources);
Map<String, DataSource> modifiedDataSources = getModifiedDataSources(oldSchemaContext, newDataSources);
oldSchemaContext.getSchema().closeDataSources(deletedDataSources);
......@@ -346,7 +346,7 @@ public class GovernanceSchemaContexts implements SchemaContexts {
return DataSourceConverter.getDataSourceMap(modifiedDataSourceConfigs);
}
private synchronized boolean isModifiedDataSource(final Map<String, DataSource> oldDataSources, final String newDataSourceName, final DataSourceConfiguration newDataSourceConfig) {
private boolean isModifiedDataSource(final Map<String, DataSource> oldDataSources, final String newDataSourceName, final DataSourceConfiguration newDataSourceConfig) {
DataSourceConfiguration dataSourceConfig = DataSourceConverter.getDataSourceConfigurationMap(oldDataSources).get(newDataSourceName);
return newDataSourceConfig.equals(dataSourceConfig);
}
......
......@@ -50,10 +50,9 @@ import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.Map;
import java.util.Properties;
......@@ -149,7 +148,7 @@ public final class GovernanceSchemaContextsTest {
@Test
@SneakyThrows(Exception.class)
public void assertSchemaAdd() {
SchemaAddedEvent event = new SchemaAddedEvent("schema_add", getDataSourceConfigurations(), new ArrayList<>());
SchemaAddedEvent event = new SchemaAddedEvent("schema_add", getDataSourceConfigurations(), new LinkedList<>());
governanceSchemaContexts.renew(event);
assertNotNull(governanceSchemaContexts.getSchemaContexts().get("schema_add"));
}
......@@ -160,7 +159,7 @@ public final class GovernanceSchemaContextsTest {
dataSource.setUrl("jdbc:h2:mem:test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL");
dataSource.setUsername("sa");
dataSource.setPassword("");
Map<String, DataSourceConfiguration> result = new LinkedHashMap<>();
Map<String, DataSourceConfiguration> result = new LinkedHashMap<>(3, 1);
result.put("ds_m", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
result.put("ds_0", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
result.put("ds_1", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
......@@ -193,15 +192,15 @@ public final class GovernanceSchemaContextsTest {
@Test
public void assertMetaDataChanged() {
MetaDataChangedEvent event = new MetaDataChangedEvent(Arrays.asList("schema_changed"), mock(RuleSchemaMetaData.class));
MetaDataChangedEvent event = new MetaDataChangedEvent(Collections.singletonList("schema_changed"), mock(RuleSchemaMetaData.class));
governanceSchemaContexts.renew(event);
assertTrue(governanceSchemaContexts.getSchemaContexts().keySet().contains("schema"));
assertFalse(governanceSchemaContexts.getSchemaContexts().keySet().contains("schema_changed"));
assertTrue(governanceSchemaContexts.getSchemaContexts().containsKey("schema"));
assertFalse(governanceSchemaContexts.getSchemaContexts().containsKey("schema_changed"));
}
@Test
public void assertMetaDataChangedWithExistSchema() {
MetaDataChangedEvent event = new MetaDataChangedEvent(Arrays.asList("schema"), mock(RuleSchemaMetaData.class));
MetaDataChangedEvent event = new MetaDataChangedEvent(Collections.singletonList("schema"), mock(RuleSchemaMetaData.class));
governanceSchemaContexts.renew(event);
assertThat(governanceSchemaContexts.getSchemaContexts().get("schema"), not(schemaContext));
}
......@@ -210,7 +209,7 @@ public final class GovernanceSchemaContextsTest {
@SneakyThrows(Exception.class)
public void assertRuleConfigurationsChanged() {
assertThat(governanceSchemaContexts.getSchemaContexts().get("schema"), is(schemaContext));
RuleConfigurationsChangedEvent event = new RuleConfigurationsChangedEvent("schema", new ArrayList<>());
RuleConfigurationsChangedEvent event = new RuleConfigurationsChangedEvent("schema", new LinkedList<>());
governanceSchemaContexts.renew(event);
assertThat(governanceSchemaContexts.getSchemaContexts().get("schema"), not(schemaContext));
}
......@@ -227,7 +226,7 @@ public final class GovernanceSchemaContextsTest {
public void assertDataSourceChanged() {
DataSourceChangedEvent event = new DataSourceChangedEvent("schema", getChangedDataSourceConfigurations());
governanceSchemaContexts.renew(event);
assertTrue(governanceSchemaContexts.getSchemaContexts().get("schema").getSchema().getDataSources().keySet().contains("ds_2"));
assertTrue(governanceSchemaContexts.getSchemaContexts().get("schema").getSchema().getDataSources().containsKey("ds_2"));
}
private Map<String, DataSourceConfiguration> getChangedDataSourceConfigurations() {
......@@ -236,7 +235,7 @@ public final class GovernanceSchemaContextsTest {
dataSource.setUrl("jdbc:h2:mem:test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL");
dataSource.setUsername("sa");
dataSource.setPassword("");
Map<String, DataSourceConfiguration> result = new LinkedHashMap<>();
Map<String, DataSourceConfiguration> result = new LinkedHashMap<>(3, 1);
result.put("ds_m", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
result.put("ds_1", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
result.put("ds_2", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
......
......@@ -21,16 +21,16 @@ import com.google.common.base.Preconditions;
import lombok.Getter;
import lombok.Setter;
import org.apache.shardingsphere.driver.governance.internal.circuit.datasource.CircuitBreakerDataSource;
import org.apache.shardingsphere.driver.governance.internal.schema.JDBCGovernanceSchemaContexts;
import org.apache.shardingsphere.driver.jdbc.core.connection.ShardingSphereConnection;
import org.apache.shardingsphere.driver.jdbc.unsupported.AbstractUnsupportedOperationDataSource;
import org.apache.shardingsphere.governance.core.config.ConfigCenter;
import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
import org.apache.shardingsphere.governance.core.schema.GovernanceSchemaContexts;
import org.apache.shardingsphere.governance.repository.api.config.GovernanceConfiguration;
import org.apache.shardingsphere.infra.auth.Authentication;
import org.apache.shardingsphere.infra.config.RuleConfiguration;
import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
import org.apache.shardingsphere.infra.config.datasource.DataSourceConverter;
import org.apache.shardingsphere.infra.config.RuleConfiguration;
import org.apache.shardingsphere.infra.context.SchemaContexts;
import org.apache.shardingsphere.infra.context.SchemaContextsBuilder;
import org.apache.shardingsphere.infra.database.DefaultSchema;
......@@ -68,14 +68,14 @@ public final class GovernanceShardingSphereDataSource extends AbstractUnsupporte
public GovernanceShardingSphereDataSource(final GovernanceConfiguration governanceConfig) throws SQLException {
GovernanceFacade governanceFacade = createGovernanceFacade(governanceConfig);
schemaContexts = new JDBCGovernanceSchemaContexts(createSchemaContexts(governanceFacade), governanceFacade);
schemaContexts = new GovernanceSchemaContexts(createSchemaContexts(governanceFacade), governanceFacade);
transactionContexts = createTransactionContexts(schemaContexts.getDatabaseType(), schemaContexts.getDefaultSchemaContext().getSchema().getDataSources());
}
public GovernanceShardingSphereDataSource(final Map<String, DataSource> dataSourceMap, final Collection<RuleConfiguration> ruleConfigurations,
final Properties props, final GovernanceConfiguration governanceConfig) throws SQLException {
GovernanceFacade governanceFacade = createGovernanceFacade(governanceConfig);
schemaContexts = new JDBCGovernanceSchemaContexts(createSchemaContexts(dataSourceMap, ruleConfigurations, props), governanceFacade);
schemaContexts = new GovernanceSchemaContexts(createSchemaContexts(dataSourceMap, ruleConfigurations, props), governanceFacade);
transactionContexts = createTransactionContexts(schemaContexts.getDatabaseType(), schemaContexts.getDefaultSchemaContext().getSchema().getDataSources());
uploadLocalConfiguration(governanceFacade);
}
......
/*
* 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.driver.governance.internal.schema;
import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
import org.apache.shardingsphere.governance.core.schema.GovernanceSchemaContexts;
import org.apache.shardingsphere.infra.context.SchemaContexts;
/**
* JDBC governance schemaContexts.
*/
public final class JDBCGovernanceSchemaContexts extends GovernanceSchemaContexts {
public JDBCGovernanceSchemaContexts(final SchemaContexts schemaContexts, final GovernanceFacade governanceFacade) {
super(schemaContexts, governanceFacade);
}
}
......@@ -20,17 +20,17 @@ package org.apache.shardingsphere.driver.governance.internal.datasource;
import com.google.common.collect.ImmutableMap;
import org.apache.commons.dbcp2.BasicDataSource;
import org.apache.shardingsphere.driver.api.yaml.YamlShardingSphereDataSourceFactory;
import org.apache.shardingsphere.driver.governance.internal.schema.JDBCGovernanceSchemaContexts;
import org.apache.shardingsphere.driver.jdbc.core.datasource.ShardingSphereDataSource;
import org.apache.shardingsphere.governance.core.common.event.datasource.DataSourceChangedEvent;
import org.apache.shardingsphere.governance.core.common.event.props.PropertiesChangedEvent;
import org.apache.shardingsphere.governance.core.common.event.rule.RuleConfigurationsChangedEvent;
import org.apache.shardingsphere.governance.core.registry.event.DisabledStateChangedEvent;
import org.apache.shardingsphere.governance.core.registry.schema.GovernanceSchema;
import org.apache.shardingsphere.governance.core.schema.GovernanceSchemaContexts;
import org.apache.shardingsphere.governance.repository.api.config.GovernanceCenterConfiguration;
import org.apache.shardingsphere.governance.repository.api.config.GovernanceConfiguration;
import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
import org.apache.shardingsphere.infra.context.SchemaContexts;
import org.apache.shardingsphere.infra.database.DefaultSchema;
import org.apache.shardingsphere.masterslave.api.config.MasterSlaveRuleConfiguration;
......@@ -58,14 +58,14 @@ import static org.junit.Assert.assertThat;
public final class GovernanceShardingSphereDataSourceTest {
private static JDBCGovernanceSchemaContexts governanceSchemaContexts;
private static GovernanceSchemaContexts governanceSchemaContexts;
@BeforeClass
public static void setUp() throws SQLException, IOException, URISyntaxException {
SchemaContexts schemaContexts = getShardingSphereDataSource().getSchemaContexts();
GovernanceShardingSphereDataSource governanceDataSource = new GovernanceShardingSphereDataSource(schemaContexts.getDefaultSchemaContext().getSchema().getDataSources(),
schemaContexts.getDefaultSchemaContext().getSchema().getConfigurations(), schemaContexts.getProps().getProps(), getGovernanceConfiguration());
governanceSchemaContexts = (JDBCGovernanceSchemaContexts) governanceDataSource.getSchemaContexts();
governanceSchemaContexts = (GovernanceSchemaContexts) governanceDataSource.getSchemaContexts();
}
private static ShardingSphereDataSource getShardingSphereDataSource() throws IOException, SQLException, URISyntaxException {
......@@ -91,12 +91,11 @@ public final class GovernanceShardingSphereDataSourceTest {
@Test
public void assertInitializeGovernanceShardingSphereDataSource() throws SQLException {
GovernanceShardingSphereDataSource governanceShardingSphereDataSource = new GovernanceShardingSphereDataSource(getGovernanceConfiguration());
assertThat(governanceShardingSphereDataSource.getConnection(), instanceOf(Connection.class));
assertThat(new GovernanceShardingSphereDataSource(getGovernanceConfiguration()).getConnection(), instanceOf(Connection.class));
}
@Test
public void assertRenewRules() throws Exception {
public void assertRenewRules() throws SQLException {
governanceSchemaContexts.renew(new RuleConfigurationsChangedEvent(DefaultSchema.LOGIC_NAME, Arrays.asList(getShardingRuleConfiguration(), getMasterSlaveRuleConfiguration())));
assertThat(((ShardingRule) governanceSchemaContexts.getDefaultSchemaContext().getSchema().getRules().iterator().next()).getTableRules().size(), is(1));
}
......@@ -126,7 +125,7 @@ public final class GovernanceShardingSphereDataSourceTest {
dataSource.setUrl("jdbc:h2:mem:test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL");
dataSource.setUsername("sa");
dataSource.setPassword("");
Map<String, DataSourceConfiguration> result = new LinkedHashMap<>();
Map<String, DataSourceConfiguration> result = new LinkedHashMap<>(3, 1);
result.put("ds_m", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
result.put("ds_s", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
result.put("ds_0", DataSourceConfiguration.getDataSourceConfiguration(dataSource));
......
/*
* 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.driver.governance.internal.schema;
import org.apache.commons.dbcp2.BasicDataSource;
import org.apache.shardingsphere.governance.core.common.event.datasource.DataSourceChangedEvent;
import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
import org.apache.shardingsphere.infra.auth.Authentication;
import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
import org.apache.shardingsphere.infra.config.datasource.DataSourceConverter;
import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
import org.apache.shardingsphere.infra.context.SchemaContext;
import org.apache.shardingsphere.infra.context.impl.StandardSchemaContexts;
import org.apache.shardingsphere.infra.context.runtime.RuntimeContext;
import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
import org.apache.shardingsphere.infra.database.DefaultSchema;
import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;
import javax.sql.DataSource;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Properties;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@RunWith(MockitoJUnitRunner.class)
public final class JDBCGovernanceSchemaContextsTest {
@Mock
private GovernanceFacade facade;
private JDBCGovernanceSchemaContexts schemaContexts;
@Before
public void setUp() {
schemaContexts = new JDBCGovernanceSchemaContexts(new StandardSchemaContexts(new LinkedHashMap<>(), new Authentication(),
new ConfigurationProperties(new Properties()), new H2DatabaseType()), facade);
}
private Map<String, SchemaContext> getSchemaContextMap() {
ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
when(schema.getDataSources()).thenReturn(getDataSources());
RuntimeContext runtimeContext = mock(RuntimeContext.class);
SchemaContext result = new SchemaContext(DefaultSchema.LOGIC_NAME, schema, runtimeContext);
return Collections.singletonMap(DefaultSchema.LOGIC_NAME, result);
}
private Map<String, DataSource> getDataSources() {
BasicDataSource result = new BasicDataSource();
result.setDriverClassName("org.h2.Driver");
result.setUrl("jdbc:h2:mem:test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL");
result.setUsername("sa");
result.setPassword("");
return Collections.singletonMap("db0", result);
}
@Test
public void assertRenewDataSourceChangedEvent() throws Exception {
schemaContexts.getSchemaContexts().putAll(getSchemaContextMap());
DataSourceChangedEvent event = new DataSourceChangedEvent(DefaultSchema.LOGIC_NAME, getDataSourceConfigurations());
schemaContexts.renew(event);
}
private Map<String, DataSourceConfiguration> getDataSourceConfigurations() {
BasicDataSource result = new BasicDataSource();
result.setDriverClassName("org.h2.Driver");
result.setUrl("jdbc:h2:mem:test1;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL");
result.setUsername("sa");
result.setPassword("");
return DataSourceConverter.getDataSourceConfigurationMap(Collections.singletonMap("db0", result));
}
}
......@@ -22,6 +22,7 @@ import lombok.NoArgsConstructor;
import lombok.extern.slf4j.Slf4j;
import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLServerInfo;
import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
import org.apache.shardingsphere.governance.core.schema.GovernanceSchemaContexts;
import org.apache.shardingsphere.governance.core.transaction.GovernanceTransactionContexts;
import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
import org.apache.shardingsphere.infra.context.SchemaContext;
......@@ -37,7 +38,6 @@ import org.apache.shardingsphere.proxy.config.yaml.swapper.YamlProxyConfiguratio
import org.apache.shardingsphere.proxy.db.DatabaseServerInfo;
import org.apache.shardingsphere.proxy.frontend.bootstrap.ShardingSphereProxy;
import org.apache.shardingsphere.proxy.governance.GovernanceBootstrap;
import org.apache.shardingsphere.proxy.governance.schema.ProxyGovernanceSchemaContexts;
import org.apache.shardingsphere.tracing.opentracing.OpenTracingTracer;
import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
import org.apache.shardingsphere.transaction.context.TransactionContexts;
......@@ -95,7 +95,7 @@ public final class Bootstrap {
}
private static SchemaContexts createSchemaContexts(final SchemaContexts schemaContexts, final boolean governanceEnabled) {
return governanceEnabled ? new ProxyGovernanceSchemaContexts(schemaContexts, GovernanceFacade.getInstance()) : schemaContexts;
return governanceEnabled ? new GovernanceSchemaContexts(schemaContexts, GovernanceFacade.getInstance()) : schemaContexts;
}
private static TransactionContexts createTransactionContexts(final SchemaContexts schemaContexts, final boolean governanceEnabled) {
......
/*
* 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.proxy.governance.schema;
import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
import org.apache.shardingsphere.governance.core.schema.GovernanceSchemaContexts;
import org.apache.shardingsphere.infra.context.SchemaContexts;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCBackendDataSourceFactory;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCRawBackendDataSourceFactory;
/**
* Proxy governance schema contexts.
*/
public final class ProxyGovernanceSchemaContexts extends GovernanceSchemaContexts {
private final JDBCBackendDataSourceFactory backendDataSourceFactory;
public ProxyGovernanceSchemaContexts(final SchemaContexts schemaContexts, final GovernanceFacade governanceFacade) {
super(schemaContexts, governanceFacade);
backendDataSourceFactory = JDBCRawBackendDataSourceFactory.getInstance();
}
}
/*
* 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.proxy.governance.schema;
import lombok.SneakyThrows;
import org.apache.shardingsphere.governance.core.common.event.auth.AuthenticationChangedEvent;
import org.apache.shardingsphere.governance.core.common.event.props.PropertiesChangedEvent;
import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
import org.apache.shardingsphere.governance.core.registry.event.CircuitStateChangedEvent;
import org.apache.shardingsphere.infra.auth.Authentication;
import org.apache.shardingsphere.infra.auth.ProxyUser;
import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
import org.apache.shardingsphere.infra.context.SchemaContext;
import org.apache.shardingsphere.infra.context.SchemaContexts;
import org.apache.shardingsphere.infra.context.impl.StandardSchemaContexts;
import org.apache.shardingsphere.infra.context.runtime.RuntimeContext;
import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;
import java.lang.reflect.Field;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
@RunWith(MockitoJUnitRunner.class)
public final class ProxyGovernanceSchemaContextsTest {
@Mock
private GovernanceFacade governanceFacade;
@Before
@SneakyThrows(ReflectiveOperationException.class)
public void setUp() {
Field field = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
field.setAccessible(true);
field.set(ProxySchemaContexts.getInstance(), getProxyGovernanceSchemaContexts());
}
private Map<String, SchemaContext> getSchemaContextMap() {
Map<String, SchemaContext> result = new HashMap<>(10);
for (int i = 0; i < 10; i++) {
String name = "schema_" + i;
ShardingSphereSchema schema = mock(ShardingSphereSchema.class);
RuntimeContext runtimeContext = mock(RuntimeContext.class);
result.put(name, new SchemaContext(name, schema, runtimeContext));
}
return result;
}
private ProxyGovernanceSchemaContexts getProxyGovernanceSchemaContexts() {
ProxyGovernanceSchemaContexts result = new ProxyGovernanceSchemaContexts(new StandardSchemaContexts(), governanceFacade);
SchemaContexts schemaContexts =
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType());
result.getSchemaContexts().putAll(schemaContexts.getSchemaContexts());
return result;
}
@Test
public void assertRenewProperties() {
assertTrue(ProxySchemaContexts.getInstance().getSchemaContexts().getProps().getProps().isEmpty());
Properties props = new Properties();
props.setProperty(ConfigurationPropertyKey.SQL_SHOW.getKey(), Boolean.TRUE.toString());
ShardingSphereEventBus.getInstance().post(new PropertiesChangedEvent(props));
assertFalse(ProxySchemaContexts.getInstance().getSchemaContexts().getProps().getProps().isEmpty());
}
@Test
public void assertRenewAuthentication() {
ProxyUser proxyUser = new ProxyUser("root", Collections.singleton("db1"));
Authentication authentication = new Authentication();
authentication.getUsers().put("root", proxyUser);
ShardingSphereEventBus.getInstance().post(new AuthenticationChangedEvent(authentication));
assertThat(ProxySchemaContexts.getInstance().getSchemaContexts().getAuthentication().getUsers().keySet().iterator().next(), is("root"));
assertThat(ProxySchemaContexts.getInstance().getSchemaContexts().getAuthentication().getUsers().get("root").getPassword(), is("root"));
assertThat(ProxySchemaContexts.getInstance().getSchemaContexts().getAuthentication().getUsers().get("root").getAuthorizedSchemas().iterator().next(), is("db1"));
}
@Test
public void assertRenewCircuitState() {
assertFalse(ProxySchemaContexts.getInstance().getSchemaContexts().isCircuitBreak());
ShardingSphereEventBus.getInstance().post(new CircuitStateChangedEvent(true));
assertTrue(ProxySchemaContexts.getInstance().getSchemaContexts().isCircuitBreak());
ShardingSphereEventBus.getInstance().post(new CircuitStateChangedEvent(false));
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册