Skip to content

Commit

Permalink
Merge pull request apache#1081 from tristaZero/dev
Browse files Browse the repository at this point in the history
Add test cases for Proxy and fix some bugs
  • Loading branch information
terrymanu authored Aug 1, 2018
2 parents 12cf09b + 147c43c commit 66f1d75
Show file tree
Hide file tree
Showing 30 changed files with 691 additions and 154 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package io.shardingsphere.core.merger.dal.show;

import com.google.common.base.Optional;
import io.shardingsphere.core.constant.DatabaseType;
import io.shardingsphere.core.merger.QueryResult;
import io.shardingsphere.core.merger.dql.common.MemoryMergedResult;
Expand Down Expand Up @@ -63,20 +62,17 @@ private Iterator<MemoryQueryResultRow> init(final List<QueryResult> queryResults
while (each.next()) {
MemoryQueryResultRow memoryResultSetRow = new MemoryQueryResultRow(each);
String tableName = memoryResultSetRow.getCell(1).toString();
Optional<TableRule> tableRule = shardingRule.tryFindTableRuleByActualTable(tableName);
// TODO :panjuan add default datasource
if (tableRule.isPresent()) {
String logicTableName = tableRule.get().getLogicTable();
memoryResultSetRow.setCell(1, logicTableName);
String createTableDDL = memoryResultSetRow.getCell(2).toString();
SQLParsingEngine sqlParsingEngine = new SQLParsingEngine(DatabaseType.MySQL, createTableDDL, shardingRule, null);
String actualTableName = sqlParsingEngine.parse(true).getTables().getSingleTableName();
if (actualTableName.startsWith("`")) {
logicTableName = "`" + logicTableName + "`";
}
memoryResultSetRow.setCell(2, createTableDDL.replaceFirst(actualTableName, logicTableName));
result.add(memoryResultSetRow);
TableRule tableRule = shardingRule.getTableRuleByActualTableName(tableName);
String logicTableName = tableRule.getLogicTable();
memoryResultSetRow.setCell(1, logicTableName);
String createTableDDL = memoryResultSetRow.getCell(2).toString();
SQLParsingEngine sqlParsingEngine = new SQLParsingEngine(DatabaseType.MySQL, createTableDDL, shardingRule, null);
String actualTableName = sqlParsingEngine.parse(true).getTables().getSingleTableName();
if (actualTableName.startsWith("`")) {
logicTableName = "`" + logicTableName + "`";
}
memoryResultSetRow.setCell(2, createTableDDL.replaceFirst(actualTableName, logicTableName));
result.add(memoryResultSetRow);
}
}
if (!result.isEmpty()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
*
* @author panjuan
*/
public class ShardingDataSourceMetaData {
public final class ShardingDataSourceMetaData {

private final Map<String, DataSourceMetaData> dataSourceMetaDataMap;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ public final class TableMetaDataLoader {
* @param shardingRule sharding rule
*/
public TableMetaData loadTableMetaData(final String logicTableName, final ShardingRule shardingRule) {
return loadTableMetaData(shardingRule.getTableRule(logicTableName), shardingRule.getShardingDataSourceNames());
return loadTableMetaData(shardingRule.getTableRuleByLogicTableName(logicTableName), shardingRule.getShardingDataSourceNames());
}

private TableMetaData loadTableMetaData(final TableRule tableRule, final ShardingDataSourceNames shardingDataSourceNames) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ private IndexToken getIndexToken() {

private Collection<TableUnit> getAllTableUnits(final String logicTableName) {
Collection<TableUnit> result = new LinkedList<>();
TableRule tableRule = shardingRule.getTableRule(logicTableName);
TableRule tableRule = shardingRule.getTableRuleByLogicTableName(logicTableName);
for (DataNode each : tableRule.getActualDataNodes()) {
TableUnit tableUnit = new TableUnit(each.getDataSourceName());
tableUnit.getRoutingTables().add(new RoutingTable(logicTableName, each.getTableName()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public final class StandardRoutingEngine implements RoutingEngine {

@Override
public RoutingResult route() {
TableRule tableRule = shardingRule.getTableRule(logicTableName);
TableRule tableRule = shardingRule.getTableRuleByLogicTableName(logicTableName);
Collection<String> databaseShardingColumns = shardingRule.getDatabaseShardingStrategy(tableRule).getShardingColumns();
Collection<String> tableShardingColumns = shardingRule.getTableShardingStrategy(tableRule).getShardingColumns();
Collection<DataNode> routedDataNodes = new LinkedHashSet<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public ShardingRule(final ShardingRuleConfiguration shardingRuleConfig, final Co
for (String group : shardingRuleConfig.getBindingTableGroups()) {
List<TableRule> tableRulesForBinding = new LinkedList<>();
for (String logicTableNameForBindingTable : StringUtil.splitWithComma(group)) {
tableRulesForBinding.add(getTableRule(logicTableNameForBindingTable));
tableRulesForBinding.add(getTableRuleByLogicTableName(logicTableNameForBindingTable));
}
bindingTableRules.add(new BindingTableRule(tableRulesForBinding));
}
Expand Down Expand Up @@ -121,13 +121,30 @@ public Optional<TableRule> tryFindTableRuleByActualTable(final String actualTabl
return Optional.absent();
}

/**
* Find table rule though actual table name.
*
* @param actualTableName actual table name
* @return table rule
*/
public TableRule getTableRuleByActualTableName(final String actualTableName) {
Optional<TableRule> tableRule = tryFindTableRuleByActualTable(actualTableName.toLowerCase());
if (tableRule.isPresent()) {
return tableRule.get();
}
if (!Strings.isNullOrEmpty(shardingDataSourceNames.getDefaultDataSourceName())) {
return createTableRuleWithDefaultDataSource(actualTableName.toLowerCase());
}
throw new ShardingConfigurationException("Cannot find table rule and default data source with actual table: '%s'", actualTableName);
}

/**
* Find table rule though logic table name.
*
* @param logicTableName logic table name
* @return table rule
*/
public TableRule getTableRule(final String logicTableName) {
public TableRule getTableRuleByLogicTableName(final String logicTableName) {
Optional<TableRule> tableRule = tryFindTableRuleByLogicTable(logicTableName.toLowerCase());
if (tableRule.isPresent()) {
return tableRule.get();
Expand Down Expand Up @@ -321,7 +338,7 @@ public DataNode findDataNode(final String logicTableName) {
* @return data node
*/
public DataNode findDataNode(final String dataSourceName, final String logicTableName) {
TableRule tableRule = getTableRule(logicTableName);
TableRule tableRule = getTableRuleByLogicTableName(logicTableName);
for (DataNode each : tableRule.getActualDataNodes()) {
if (shardingDataSourceNames.getDataSourceNames().contains(each.getDataSourceName()) && (null == dataSourceName || each.getDataSourceName().equals(dataSourceName))) {
return each;
Expand All @@ -342,7 +359,7 @@ public DataNode findDataNode(final String dataSourceName, final String logicTabl
* @return is logic index or not
*/
public boolean isLogicIndex(final String logicIndexName, final String logicTableName) {
return logicIndexName.equals(getTableRule(logicTableName).getLogicIndex());
return logicIndexName.equals(getTableRuleByLogicTableName(logicTableName).getLogicIndex());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,11 +106,4 @@ public void assertNextForTableRuleIsPresentForNoBackQuotes() throws SQLException
ShowCreateTableMergedResult showCreateTableMergedResult = new ShowCreateTableMergedResult(shardingRule, queryResults);
assertTrue(showCreateTableMergedResult.next());
}

@Test
public void assertNextForTableRuleIsNotPresent() throws SQLException {
when(resultSet.getObject(1)).thenReturn("table_3");
ShowCreateTableMergedResult showCreateTableMergedResult = new ShowCreateTableMergedResult(shardingRule, queryResults);
assertFalse(showCreateTableMergedResult.next());
}
}
Original file line number Diff line number Diff line change
@@ -1,3 +1,20 @@
/*
* Copyright 2016-2018 shardingsphere.io.
* <p>
* Licensed 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.
* </p>
*/

package io.shardingsphere.core.metadata.datasource;

import io.shardingsphere.core.constant.DatabaseType;
Expand All @@ -8,32 +25,33 @@
import io.shardingsphere.core.metadata.datasource.dialect.SQLServerDataSourceMetaData;
import org.junit.Test;

import static org.junit.Assert.assertTrue;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.junit.Assert.assertThat;

public class DataSourceMetaDataFactoryTest {

@Test
public void assertAllNewInstanceForH2() {
assertTrue((DataSourceMetaDataFactory.newInstance(DatabaseType.H2, "jdbc:h2:mem:ds_0;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL") instanceof H2DataSourceMetaData));
assertThat(DataSourceMetaDataFactory.newInstance(DatabaseType.H2, "jdbc:h2:mem:ds_0;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL"), instanceOf(H2DataSourceMetaData.class));
}

@Test
public void assertAllNewInstanceForMySQL() {
assertTrue((DataSourceMetaDataFactory.newInstance(DatabaseType.MySQL, "jdbc:mysql://127.0.0.1:3306/db_0") instanceof MySQLDataSourceMetaData));
assertThat(DataSourceMetaDataFactory.newInstance(DatabaseType.MySQL, "jdbc:mysql://127.0.0.1:3306/db_0"), instanceOf(MySQLDataSourceMetaData.class));
}

@Test
public void assertAllNewInstanceForOracle() {
assertTrue((DataSourceMetaDataFactory.newInstance(DatabaseType.Oracle, "jdbc:oracle:thin:@//127.0.0.1:3306/ds_0") instanceof OracleDataSourceMetaData));
assertThat(DataSourceMetaDataFactory.newInstance(DatabaseType.Oracle, "jdbc:oracle:thin:@//127.0.0.1:3306/ds_0"), instanceOf(OracleDataSourceMetaData.class));
}

@Test
public void assertAllNewInstanceForPostgreSQL() {
assertTrue((DataSourceMetaDataFactory.newInstance(DatabaseType.PostgreSQL, "jdbc:postgresql://127.0.0.1:3306/ds_0") instanceof PostgreSQLDataSourceMetaData));
assertThat(DataSourceMetaDataFactory.newInstance(DatabaseType.PostgreSQL, "jdbc:postgresql://127.0.0.1:3306/ds_0"), instanceOf(PostgreSQLDataSourceMetaData.class));
}

@Test
public void assertAllNewInstanceForSQLServer() {
assertTrue((DataSourceMetaDataFactory.newInstance(DatabaseType.SQLServer, "jdbc:microsoft:sqlserver://127.0.0.1:3306;DatabaseName=ds_0") instanceof SQLServerDataSourceMetaData));
assertThat(DataSourceMetaDataFactory.newInstance(DatabaseType.SQLServer, "jdbc:microsoft:sqlserver://127.0.0.1:3306;DatabaseName=ds_0"), instanceOf(SQLServerDataSourceMetaData.class));
}
}
}
Original file line number Diff line number Diff line change
@@ -1,3 +1,20 @@
/*
* Copyright 2016-2018 shardingsphere.io.
* <p>
* Licensed 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.
* </p>
*/

package io.shardingsphere.core.metadata.datasource;

import com.google.common.collect.Lists;
Expand All @@ -9,7 +26,7 @@
import org.junit.Before;
import org.junit.Test;

import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.Map;

Expand All @@ -23,28 +40,35 @@ public class ShardingDataSourceMetaDataTest {

@Before
public void setUp() {
Map<String, String> masterSlaveShardingDataSourceURLs = new LinkedHashMap<String, String>() {{ put("single", "jdbc:mysql://127.0.0.1:3306/single"); put("master_0", "jdbc:mysql://127.0.0.1:3306/master_0");
put("master_1", "jdbc:mysql://127.0.0.1:3306/master_1"); put("master_2", "jdbc:mysql://127.0.0.1:3307/master_2"); put("slave_0", "jdbc:mysql://127.0.0.2:3306/slave_0");
put("slave_1", "jdbc:mysql://127.0.0.2:3306/slave_1"); put("slave_2", "jdbc:mysql://127.0.0.2:3307/slave_2");}};
Map<String, String> masterSlaveShardingDataSourceURLs = new LinkedHashMap<>();
masterSlaveShardingDataSourceURLs.put("single", "jdbc:mysql://127.0.0.1:3306/single");
masterSlaveShardingDataSourceURLs.put("master_0", "jdbc:mysql://127.0.0.1:3306/master_0");
masterSlaveShardingDataSourceURLs.put("master_1", "jdbc:mysql://127.0.0.1:3306/master_1");
masterSlaveShardingDataSourceURLs.put("master_2", "jdbc:mysql://127.0.0.1:3307/master_2");
masterSlaveShardingDataSourceURLs.put("slave_0", "jdbc:mysql://127.0.0.2:3306/slave_0");
masterSlaveShardingDataSourceURLs.put("slave_1", "jdbc:mysql://127.0.0.2:3306/slave_1");
masterSlaveShardingDataSourceURLs.put("slave_2", "jdbc:mysql://127.0.0.2:3307/slave_2");
masterSlaveShardingDataSourceMetaData = new ShardingDataSourceMetaData(masterSlaveShardingDataSourceURLs, getMasterSlaveShardingRule(), DatabaseType.MySQL);
Map<String, String> shardingDataSourceURLs = new LinkedHashMap<String, String>() {{ put("ds_0", "jdbc:mysql://127.0.0.1:3306/db_0"); put("ds_1", "jdbc:mysql://127.0.0.1:3306/db_1"); }};
Map<String, String> shardingDataSourceURLs = new LinkedHashMap<>();
shardingDataSourceURLs.put("ds_0", "jdbc:mysql://127.0.0.1:3306/db_0");
shardingDataSourceURLs.put("ds_1", "jdbc:mysql://127.0.0.1:3306/db_1");
shardingDataSourceMetaData = new ShardingDataSourceMetaData(shardingDataSourceURLs, getShardingRule(), DatabaseType.MySQL);
}

private ShardingRule getMasterSlaveShardingRule() {
TableRuleConfiguration tableRuleConfig_0 = new TableRuleConfiguration();
tableRuleConfig_0.setLogicTable("t_order");
tableRuleConfig_0.setActualDataNodes("ms_${0..2}.t_order_${0..1}");
TableRuleConfiguration tableRuleConfig_1 = new TableRuleConfiguration();
tableRuleConfig_1.setLogicTable("t_order_item");
tableRuleConfig_1.setActualDataNodes("single.t_order_item");
TableRuleConfiguration tableRuleConfig0 = new TableRuleConfiguration();
tableRuleConfig0.setLogicTable("t_order");
tableRuleConfig0.setActualDataNodes("ms_${0..2}.t_order_${0..1}");
TableRuleConfiguration tableRuleConfig1 = new TableRuleConfiguration();
tableRuleConfig1.setLogicTable("t_order_item");
tableRuleConfig1.setActualDataNodes("single.t_order_item");
ShardingRuleConfiguration shardingRuleConfig = new ShardingRuleConfiguration();
MasterSlaveRuleConfiguration MasterSlaveConfig_0 = new MasterSlaveRuleConfiguration("ms_0", "master_0", Arrays.asList("slave_0"));
MasterSlaveRuleConfiguration MasterSlaveConfig_1 = new MasterSlaveRuleConfiguration("ms_1", "master_1", Arrays.asList("slave_1"));
MasterSlaveRuleConfiguration MasterSlaveConfig_2 = new MasterSlaveRuleConfiguration("ms_2", "master_2", Arrays.asList("slave_2"));
shardingRuleConfig.getTableRuleConfigs().add(tableRuleConfig_0);
shardingRuleConfig.getTableRuleConfigs().add(tableRuleConfig_1);
shardingRuleConfig.getMasterSlaveRuleConfigs().addAll(Lists.newArrayList(MasterSlaveConfig_0, MasterSlaveConfig_1, MasterSlaveConfig_2));
MasterSlaveRuleConfiguration masterSlaveConfig0 = new MasterSlaveRuleConfiguration("ms_0", "master_0", Collections.singleton("slave_0"));
MasterSlaveRuleConfiguration masterSlaveConfig1 = new MasterSlaveRuleConfiguration("ms_1", "master_1", Collections.singleton("slave_1"));
MasterSlaveRuleConfiguration masterSlaveConfig2 = new MasterSlaveRuleConfiguration("ms_2", "master_2", Collections.singleton("slave_2"));
shardingRuleConfig.getTableRuleConfigs().add(tableRuleConfig0);
shardingRuleConfig.getTableRuleConfigs().add(tableRuleConfig1);
shardingRuleConfig.getMasterSlaveRuleConfigs().addAll(Lists.newArrayList(masterSlaveConfig0, masterSlaveConfig1, masterSlaveConfig2));
return new ShardingRule(shardingRuleConfig, Lists.newArrayList("single", "master_0", "master_1", "master_2", "slave_0", "slave_1", "slave_2"));
}

Expand Down Expand Up @@ -76,4 +100,4 @@ public void assertGetActualSchemaNameForMasterSlaveShardingRule() {
public void assertGetActualSchemaNameForShardingRule() {
assertEquals(shardingDataSourceMetaData.getActualSchemaName("ds_0"), "db_0");
}
}
}
Original file line number Diff line number Diff line change
@@ -1,3 +1,20 @@
/*
* Copyright 2016-2018 shardingsphere.io.
* <p>
* Licensed 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.
* </p>
*/

package io.shardingsphere.core.metadata.datasource.dialect;

import io.shardingsphere.core.exception.ShardingException;
Expand Down Expand Up @@ -27,4 +44,4 @@ public void assertIsInSameDatabaseInstance() {
H2DataSourceMetaData actual = new H2DataSourceMetaData("jdbc:h2:mem:ds_0;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false;MODE=MySQL");
assertThat(actual.isInSameDatabaseInstance(target), is(false));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,3 +53,4 @@ public void assertIsInSameDatabaseInstance() {
assertThat(actual.isInSameDatabaseInstance(target), is(true));
}
}

Original file line number Diff line number Diff line change
@@ -1,3 +1,20 @@
/*
* Copyright 2016-2018 shardingsphere.io.
* <p>
* Licensed 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.
* </p>
*/

package io.shardingsphere.core.metadata.datasource.dialect;

import io.shardingsphere.core.exception.ShardingException;
Expand Down Expand Up @@ -35,4 +52,4 @@ public void assertIsInSameDatabaseInstance() {
OracleDataSourceMetaData actual = new OracleDataSourceMetaData("jdbc:oracle:thin:@//127.0.0.1:1521/ds_0");
assertThat(actual.isInSameDatabaseInstance(target), is(true));
}
}
}
Loading

0 comments on commit 66f1d75

Please sign in to comment.