未验证 提交 528e9247 编写于 作者: L Liang Zhang 提交者: GitHub

Add SchemaContextsBuilderTest (#6994)

* Refactor SchemaContextsBuilder

* Add SchemaContextsBuilderTest.assertBuildWithEmptyConfiguration

* Update SchemaContextsBuilder

* Add todo

* Add SchemaContextsBuilderTest
上级 1722e874
......@@ -337,7 +337,7 @@ public abstract class OrchestrationSchemaContexts implements SchemaContexts {
Map<String, Map<String, DataSource>> dataSourcesMap = createDataSourcesMap(Collections.singletonMap(schemaName, schemaAddedEvent.getDataSourceConfigurations()));
Map<String, Map<String, DataSourceParameter>> dataSourceParametersMap = createDataSourceParametersMap(Collections.singletonMap(schemaName, schemaAddedEvent.getDataSourceConfigurations()));
DatabaseType databaseType = getDatabaseType(dataSourceParametersMap);
SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(dataSourcesMap, databaseType,
SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(databaseType, dataSourcesMap,
Collections.singletonMap(schemaName, schemaAddedEvent.getRuleConfigurations()), schemaContexts.getAuthentication(), schemaContexts.getProps().getProps());
return schemaContextsBuilder.build().getSchemaContexts().get(schemaName);
}
......@@ -360,8 +360,8 @@ public abstract class OrchestrationSchemaContexts implements SchemaContexts {
private SchemaContext getChangedSchemaContext(final SchemaContext oldSchemaContext, final Collection<RuleConfiguration> configurations) throws SQLException {
ShardingSphereSchema oldSchema = oldSchemaContext.getSchema();
SchemaContextsBuilder builder = new SchemaContextsBuilder(Collections.singletonMap(oldSchemaContext.getName(), oldSchema.getDataSources()),
schemaContexts.getDatabaseType(), Collections.singletonMap(oldSchemaContext.getName(), configurations), schemaContexts.getAuthentication(), schemaContexts.getProps().getProps());
SchemaContextsBuilder builder = new SchemaContextsBuilder(schemaContexts.getDatabaseType(), Collections.singletonMap(oldSchemaContext.getName(), oldSchema.getDataSources()),
Collections.singletonMap(oldSchemaContext.getName(), configurations), schemaContexts.getAuthentication(), schemaContexts.getProps().getProps());
return builder.build().getSchemaContexts().values().iterator().next();
}
......@@ -373,7 +373,7 @@ public abstract class OrchestrationSchemaContexts implements SchemaContexts {
oldSchemaContext.getRuntimeContext().getTransactionManagerEngine().close();
Map<String, Map<String, DataSource>> dataSourcesMap = Collections.singletonMap(oldSchemaContext.getName(), getNewDataSources(oldSchemaContext.getSchema().getDataSources(),
deletedDataSources, getAddedDataSources(oldSchemaContext, newDataSources), modifiedDataSources));
return new SchemaContextsBuilder(dataSourcesMap, schemaContexts.getDatabaseType(),
return new SchemaContextsBuilder(schemaContexts.getDatabaseType(), dataSourcesMap,
Collections.singletonMap(oldSchemaContext.getName(), oldSchemaContext.getSchema().getConfigurations()), schemaContexts.getAuthentication(),
schemaContexts.getProps().getProps()).build().getSchemaContexts().get(oldSchemaContext.getName());
}
......
......@@ -57,8 +57,8 @@ public final class ShardingSphereDataSource extends AbstractUnsupportedOperation
public ShardingSphereDataSource(final Map<String, DataSource> dataSourceMap, final Collection<RuleConfiguration> configurations, final Properties props) throws SQLException {
DatabaseType databaseType = createDatabaseType(dataSourceMap);
schemaContexts = new SchemaContextsBuilder(Collections.singletonMap(DefaultSchema.LOGIC_NAME, dataSourceMap),
databaseType, Collections.singletonMap(DefaultSchema.LOGIC_NAME, configurations), props).build();
schemaContexts = new SchemaContextsBuilder(databaseType, Collections.singletonMap(DefaultSchema.LOGIC_NAME, dataSourceMap),
Collections.singletonMap(DefaultSchema.LOGIC_NAME, configurations), props).build();
}
private DatabaseType createDatabaseType(final Map<String, DataSource> dataSourceMap) throws SQLException {
......
......@@ -49,5 +49,12 @@
<artifactId>shardingsphere-transaction-core</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.shardingsphere</groupId>
<artifactId>shardingsphere-test</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>
......@@ -46,19 +46,20 @@ import java.util.Collection;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Properties;
/**
* Schema contexts builder.
*/
@Slf4j(topic = "ShardingSphere-schemaContexts")
@Slf4j
public final class SchemaContextsBuilder {
private final DatabaseType databaseType;
private final Map<String, Map<String, DataSource>> dataSources;
private final Map<String, Collection<RuleConfiguration>> ruleConfigurations;
private final Map<String, Collection<RuleConfiguration>> ruleConfigs;
private final Authentication authentication;
......@@ -66,30 +67,30 @@ public final class SchemaContextsBuilder {
private final ExecutorKernel executorKernel;
public SchemaContextsBuilder(final Map<String, Map<String, DataSource>> dataSources,
final DatabaseType databaseType, final Map<String, Collection<RuleConfiguration>> ruleConfigurations, final Properties props) {
this(dataSources, databaseType, ruleConfigurations, new Authentication(), props);
public SchemaContextsBuilder(final DatabaseType databaseType, final Map<String, Map<String, DataSource>> dataSources,
final Map<String, Collection<RuleConfiguration>> ruleConfigs, final Properties props) {
this(databaseType, dataSources, ruleConfigs, new Authentication(), props);
}
public SchemaContextsBuilder(final Map<String, Map<String, DataSource>> dataSources,
final DatabaseType databaseType, final Map<String, Collection<RuleConfiguration>> ruleConfigurations, final Authentication authentication, final Properties props) {
this.dataSources = dataSources;
public SchemaContextsBuilder(final DatabaseType databaseType, final Map<String, Map<String, DataSource>> dataSources,
final Map<String, Collection<RuleConfiguration>> ruleConfigs, final Authentication authentication, final Properties props) {
this.databaseType = databaseType;
this.ruleConfigurations = ruleConfigurations;
this.dataSources = dataSources;
this.ruleConfigs = ruleConfigs;
this.authentication = authentication;
this.props = new ConfigurationProperties(null == props ? new Properties() : props);
executorKernel = new ExecutorKernel(this.props.<Integer>getValue(ConfigurationPropertyKey.EXECUTOR_SIZE));
}
/**
* Build.
* Build schema contexts.
*
* @exception SQLException sql exception
* @return SchemaContexts
* @exception SQLException SQL exception
* @return schema contexts
*/
public SchemaContexts build() throws SQLException {
Map<String, SchemaContext> schemaContexts = new LinkedHashMap<>();
for (String each : ruleConfigurations.keySet()) {
Map<String, SchemaContext> schemaContexts = new LinkedHashMap<>(ruleConfigs.size(), 1);
for (String each : ruleConfigs.keySet()) {
schemaContexts.put(each, createSchemaContext(each));
}
return new StandardSchemaContexts(schemaContexts, authentication, props, databaseType);
......@@ -97,17 +98,17 @@ public final class SchemaContextsBuilder {
private SchemaContext createSchemaContext(final String schemaName) throws SQLException {
Map<String, DataSource> dataSources = this.dataSources.get(schemaName);
RuntimeContext runtimeContext = new RuntimeContext(createCachedDatabaseMetaData(dataSources),
RuntimeContext runtimeContext = new RuntimeContext(createCachedDatabaseMetaData(dataSources).orElse(null),
executorKernel, ShardingSphereSQLParserEngineFactory.getSQLParserEngine(DatabaseTypes.getTrunkDatabaseTypeName(databaseType)), createShardingTransactionManagerEngine(dataSources));
return new SchemaContext(schemaName, createShardingSphereSchema(schemaName), runtimeContext);
}
private CachedDatabaseMetaData createCachedDatabaseMetaData(final Map<String, DataSource> dataSources) throws SQLException {
private Optional<CachedDatabaseMetaData> createCachedDatabaseMetaData(final Map<String, DataSource> dataSources) throws SQLException {
if (dataSources.isEmpty()) {
return null;
return Optional.empty();
}
try (Connection connection = dataSources.values().iterator().next().getConnection()) {
return new CachedDatabaseMetaData(connection.getMetaData());
return Optional.of(new CachedDatabaseMetaData(connection.getMetaData()));
}
}
......@@ -119,9 +120,9 @@ public final class SchemaContextsBuilder {
private ShardingSphereSchema createShardingSphereSchema(final String schemaName) throws SQLException {
Map<String, DataSource> dataSources = this.dataSources.get(schemaName);
Collection<RuleConfiguration> ruleConfigurations = this.ruleConfigurations.get(schemaName);
Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build(ruleConfigurations, dataSources.keySet());
return new ShardingSphereSchema(ruleConfigurations, rules, dataSources, createMetaData(dataSources, rules));
Collection<RuleConfiguration> ruleConfigs = this.ruleConfigs.get(schemaName);
Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.build(ruleConfigs, dataSources.keySet());
return new ShardingSphereSchema(ruleConfigs, rules, dataSources, createMetaData(dataSources, rules));
}
private ShardingSphereMetaData createMetaData(final Map<String, DataSource> dataSourceMap, final Collection<ShardingSphereRule> rules) throws SQLException {
......@@ -129,7 +130,7 @@ public final class SchemaContextsBuilder {
DataSourceMetas dataSourceMetas = new DataSourceMetas(databaseType, getDatabaseAccessConfigurationMap(dataSourceMap));
RuleSchemaMetaData ruleSchemaMetaData = new RuleSchemaMetaDataLoader(rules).load(databaseType, dataSourceMap, props, executorKernel.getExecutorService().getExecutorService());
ShardingSphereMetaData result = new ShardingSphereMetaData(dataSourceMetas, ruleSchemaMetaData);
log.info("Meta data load finished, cost {} milliseconds.", System.currentTimeMillis() - start);
log.info("Load meta data finished, cost {} milliseconds.", System.currentTimeMillis() - start);
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.
*/
package org.apache.shardingsphere.kernel.context;
import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
import org.apache.shardingsphere.infra.database.type.DatabaseType;
import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
import org.apache.shardingsphere.kernel.context.fixture.FixtureRule;
import org.apache.shardingsphere.kernel.context.fixture.FixtureRuleConfiguration;
import org.apache.shardingsphere.test.MockedDataSource;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.junit.MockitoJUnitRunner;
import java.sql.SQLException;
import java.util.Collections;
import java.util.Properties;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
@RunWith(MockitoJUnitRunner.class)
public final class SchemaContextsBuilderTest {
@Test
public void assertBuildWithoutConfiguration() throws SQLException {
DatabaseType databaseType = DatabaseTypes.getActualDatabaseType("FixtureDB");
SchemaContexts actual = new SchemaContextsBuilder(databaseType, Collections.emptyMap(), Collections.emptyMap(), null).build();
assertThat(actual.getDatabaseType(), is(databaseType));
assertTrue(actual.getSchemaContexts().isEmpty());
assertTrue(actual.getAuthentication().getUsers().isEmpty());
assertTrue(actual.getProps().getProps().isEmpty());
assertFalse(actual.isCircuitBreak());
}
@Test
public void assertBuildWithConfigurationsButWithoutDataSource() throws SQLException {
DatabaseType databaseType = DatabaseTypes.getActualDatabaseType("FixtureDB");
Properties props = new Properties();
props.setProperty(ConfigurationPropertyKey.EXECUTOR_SIZE.getKey(), "1");
SchemaContexts actual = new SchemaContextsBuilder(databaseType, Collections.singletonMap("logic_db", Collections.emptyMap()),
Collections.singletonMap("logic_db", Collections.singleton(new FixtureRuleConfiguration())), props).build();
assertThat(actual.getDatabaseType(), is(databaseType));
assertRules(actual);
assertTrue(actual.getSchemaContexts().get("logic_db").getSchema().getDataSources().isEmpty());
assertTrue(actual.getAuthentication().getUsers().isEmpty());
assertThat(actual.getProps().getProps().size(), is(1));
assertThat(actual.getProps().getValue(ConfigurationPropertyKey.EXECUTOR_SIZE), is(1));
assertFalse(actual.isCircuitBreak());
}
@Test
public void assertBuildWithConfigurationsAndDataSources() throws SQLException {
DatabaseType databaseType = DatabaseTypes.getActualDatabaseType("FixtureDB");
Properties props = new Properties();
props.setProperty(ConfigurationPropertyKey.EXECUTOR_SIZE.getKey(), "1");
SchemaContexts actual = new SchemaContextsBuilder(databaseType, Collections.singletonMap("logic_db", Collections.singletonMap("ds", new MockedDataSource())),
Collections.singletonMap("logic_db", Collections.singleton(new FixtureRuleConfiguration())), props).build();
assertThat(actual.getDatabaseType(), is(databaseType));
assertRules(actual);
assertDataSources(actual);
assertTrue(actual.getAuthentication().getUsers().isEmpty());
assertThat(actual.getProps().getProps().size(), is(1));
assertThat(actual.getProps().getValue(ConfigurationPropertyKey.EXECUTOR_SIZE), is(1));
assertFalse(actual.isCircuitBreak());
}
private void assertRules(final SchemaContexts actual) {
assertThat(actual.getSchemaContexts().get("logic_db").getSchema().getRules().size(), is(1));
assertThat(actual.getSchemaContexts().get("logic_db").getSchema().getRules().iterator().next(), instanceOf(FixtureRule.class));
}
private void assertDataSources(final SchemaContexts actual) {
assertThat(actual.getSchemaContexts().get("logic_db").getSchema().getDataSources().size(), is(1));
assertThat(actual.getSchemaContexts().get("logic_db").getSchema().getDataSources().get("ds"), instanceOf(MockedDataSource.class));
}
}
/*
* 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.kernel.context.fixture;
import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
import org.apache.shardingsphere.infra.database.type.DatabaseType;
import java.util.Collection;
import java.util.Collections;
import static org.mockito.Mockito.mock;
public final class FixtureDatabaseType implements DatabaseType {
@Override
public String getName() {
return "FixtureDB";
}
@Override
public Collection<String> getJdbcUrlPrefixes() {
return Collections.singleton("jdbc:fixture");
}
@Override
public DataSourceMetaData getDataSourceMetaData(final String url, final String username) {
return mock(DataSourceMetaData.class);
}
}
/*
* 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.kernel.context.fixture;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
public final class FixtureRule implements ShardingSphereRule {
}
/*
* 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.kernel.context.fixture;
import org.apache.shardingsphere.infra.rule.ShardingSphereRuleBuilder;
import java.util.Collection;
public final class FixtureRuleBuilder implements ShardingSphereRuleBuilder<FixtureRule, FixtureRuleConfiguration> {
@Override
public FixtureRule build(final FixtureRuleConfiguration ruleConfiguration, final Collection<String> dataSourceNames) {
return new FixtureRule();
}
@Override
public int getOrder() {
return 0;
}
@Override
public Class<FixtureRuleConfiguration> getTypeClass() {
return FixtureRuleConfiguration.class;
}
}
/*
* 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.kernel.context.fixture;
import org.apache.shardingsphere.infra.config.RuleConfiguration;
public final class FixtureRuleConfiguration implements RuleConfiguration {
}
#
# 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.
#
org.apache.shardingsphere.kernel.context.fixture.FixtureDatabaseType
#
# 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.
#
org.apache.shardingsphere.kernel.context.fixture.FixtureRuleBuilder
......@@ -41,6 +41,7 @@ import org.apache.shardingsphere.proxy.orchestration.OrchestrationBootstrap;
import org.apache.shardingsphere.proxy.orchestration.schema.ProxyOrchestrationSchemaContexts;
import javax.sql.DataSource;
import java.io.IOException;
import java.sql.SQLException;
import java.util.Collection;
import java.util.LinkedList;
......@@ -57,9 +58,10 @@ public final class Bootstrap {
* Main entrance.
*
* @param args startup arguments
* @throws Exception exception
* @throws IOException IO exception
* @throws SQLException SQL exception
*/
public static void main(final String[] args) throws Exception {
public static void main(final String[] args) throws IOException, SQLException {
BootstrapArguments bootstrapArgs = new BootstrapArguments(args);
int port = bootstrapArgs.getPort();
YamlProxyConfiguration yamlConfig = ProxyConfigurationLoader.load(bootstrapArgs.getConfigurationPath());
......@@ -82,7 +84,7 @@ public final class Bootstrap {
private static void initSchemaContexts(final ProxyConfiguration proxyConfig, final boolean orchestrationEnabled) throws SQLException {
ProxyDataSourceContext dataSourceContext = new ProxyDataSourceContext(proxyConfig.getSchemaDataSources());
SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(
dataSourceContext.getDataSourcesMap(), dataSourceContext.getDatabaseType(), proxyConfig.getSchemaRules(), proxyConfig.getAuthentication(), proxyConfig.getProps());
dataSourceContext.getDatabaseType(), dataSourceContext.getDataSourcesMap(), proxyConfig.getSchemaRules(), proxyConfig.getAuthentication(), proxyConfig.getProps());
ProxySchemaContexts.getInstance().init(createSchemaContexts(schemaContextsBuilder.build(), orchestrationEnabled));
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册