未验证 提交 93b8adab 编写于 作者: ShardingSphere's avatar ShardingSphere 提交者: GitHub

Redo #2601 (#3065)

* remove RoutingResultValidator

* rename org.apache.shardingsphere.core.route.router.sharding.validator.statement to org.apache.shardingsphere.core.route.router.sharding.validator

* update javadoc

* add judge to whether routed data sources belong to configured data sources
上级 70f1814a
......@@ -39,9 +39,8 @@ import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCo
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.core.route.router.sharding.condition.engine.InsertClauseShardingConditionEngine;
import org.apache.shardingsphere.core.route.router.sharding.condition.engine.WhereClauseShardingConditionEngine;
import org.apache.shardingsphere.core.route.router.sharding.validator.routingresult.RoutingResultValidatorFactory;
import org.apache.shardingsphere.core.route.router.sharding.validator.statement.ShardingStatementValidator;
import org.apache.shardingsphere.core.route.router.sharding.validator.statement.ShardingStatementValidatorFactory;
import org.apache.shardingsphere.core.route.router.sharding.validator.ShardingStatementValidator;
import org.apache.shardingsphere.core.route.router.sharding.validator.ShardingStatementValidatorFactory;
import org.apache.shardingsphere.core.route.type.RoutingEngine;
import org.apache.shardingsphere.core.route.type.RoutingResult;
import org.apache.shardingsphere.core.rule.BindingTableRule;
......@@ -107,7 +106,6 @@ public final class ShardingRouter {
}
RoutingEngine routingEngine = RoutingEngineFactory.newInstance(shardingRule, metaData, shardingStatement, shardingConditions);
RoutingResult routingResult = routingEngine.route();
RoutingResultValidatorFactory.newInstance(routingEngine, shardingRule, metaData).validate(shardingStatement, shardingConditions, routingResult);
if (needMergeShardingValues) {
Preconditions.checkState(1 == routingResult.getRoutingUnits().size(), "Must have one sharding with subquery.");
}
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.route.router.sharding.validator.statement;
package org.apache.shardingsphere.core.route.router.sharding.validator;
import org.apache.shardingsphere.core.parse.sql.statement.SQLStatement;
import org.apache.shardingsphere.core.rule.ShardingRule;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.route.router.sharding.validator.statement;
package org.apache.shardingsphere.core.route.router.sharding.validator;
import com.google.common.base.Optional;
import lombok.AccessLevel;
......@@ -23,8 +23,8 @@ import lombok.NoArgsConstructor;
import org.apache.shardingsphere.core.parse.sql.statement.SQLStatement;
import org.apache.shardingsphere.core.parse.sql.statement.dml.InsertStatement;
import org.apache.shardingsphere.core.parse.sql.statement.dml.UpdateStatement;
import org.apache.shardingsphere.core.route.router.sharding.validator.statement.impl.ShardingInsertStatementValidator;
import org.apache.shardingsphere.core.route.router.sharding.validator.statement.impl.ShardingUpdateStatementValidator;
import org.apache.shardingsphere.core.route.router.sharding.validator.impl.ShardingInsertStatementValidator;
import org.apache.shardingsphere.core.route.router.sharding.validator.impl.ShardingUpdateStatementValidator;
/**
* Sharding statement validator factory.
......
......@@ -15,18 +15,18 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.route.router.sharding.validator.statement.impl;
package org.apache.shardingsphere.core.route.router.sharding.validator.impl;
import com.google.common.base.Optional;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.parse.sql.segment.dml.column.ColumnSegment;
import org.apache.shardingsphere.core.parse.sql.segment.dml.column.OnDuplicateKeyColumnsSegment;
import org.apache.shardingsphere.core.parse.sql.statement.dml.InsertStatement;
import org.apache.shardingsphere.core.route.router.sharding.validator.statement.ShardingStatementValidator;
import org.apache.shardingsphere.core.route.router.sharding.validator.ShardingStatementValidator;
import org.apache.shardingsphere.core.rule.ShardingRule;
/**
* Sharding insert validator.
* Sharding insert statement validator.
*
* @author zhangliang
*/
......
......@@ -15,17 +15,17 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.route.router.sharding.validator.statement.impl;
package org.apache.shardingsphere.core.route.router.sharding.validator.impl;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.optimize.api.segment.Tables;
import org.apache.shardingsphere.core.parse.sql.segment.dml.assignment.AssignmentSegment;
import org.apache.shardingsphere.core.parse.sql.statement.dml.UpdateStatement;
import org.apache.shardingsphere.core.route.router.sharding.validator.statement.ShardingStatementValidator;
import org.apache.shardingsphere.core.route.router.sharding.validator.ShardingStatementValidator;
import org.apache.shardingsphere.core.rule.ShardingRule;
/**
* Sharding update validator.
* Sharding update statement validator.
*
* @author zhangliang
*/
......
/*
* 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.core.route.router.sharding.validator.routingresult;
import org.apache.shardingsphere.core.optimize.sharding.statement.ShardingOptimizedStatement;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.core.route.type.RoutingResult;
/**
* Routing result validator.
*
* @author sunbufu
* @author zhangliang
*/
public interface RoutingResultValidator {
/**
* Validate routing result.
*
* @param shardingStatement sharding statement
* @param shardingConditions sharding conditions
* @param routingResult routing result
*/
void validate(ShardingOptimizedStatement shardingStatement, ShardingConditions shardingConditions, RoutingResult routingResult);
}
/*
* 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.core.route.router.sharding.validator.routingresult;
import lombok.AccessLevel;
import lombok.NoArgsConstructor;
import org.apache.shardingsphere.core.metadata.ShardingSphereMetaData;
import org.apache.shardingsphere.core.route.router.sharding.validator.routingresult.impl.ComplexRoutingResultValidator;
import org.apache.shardingsphere.core.route.router.sharding.validator.routingresult.impl.StandardRoutingResultValidator;
import org.apache.shardingsphere.core.route.type.RoutingEngine;
import org.apache.shardingsphere.core.route.type.standard.StandardRoutingEngine;
import org.apache.shardingsphere.core.rule.ShardingRule;
/**
* Routing result validator factory.
*
* @author zhangliang
*/
@NoArgsConstructor(access = AccessLevel.PRIVATE)
public final class RoutingResultValidatorFactory {
/**
* New instance of sharding statement validator.
*
* @param routingEngine routing engine
* @param shardingRule sharding rule
* @param metaData meta data of ShardingSphere
* @return routing result validator
*/
public static RoutingResultValidator newInstance(final RoutingEngine routingEngine, final ShardingRule shardingRule, final ShardingSphereMetaData metaData) {
return routingEngine instanceof StandardRoutingEngine ? new StandardRoutingResultValidator(shardingRule, metaData) : new ComplexRoutingResultValidator(shardingRule, metaData);
}
}
/*
* 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.core.route.router.sharding.validator.routingresult.impl;
import com.google.common.base.Optional;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.core.metadata.ShardingSphereMetaData;
import org.apache.shardingsphere.core.optimize.sharding.statement.ShardingOptimizedStatement;
import org.apache.shardingsphere.core.optimize.sharding.statement.ddl.ShardingDropIndexOptimizedStatement;
import org.apache.shardingsphere.core.parse.sql.statement.dml.DMLStatement;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.core.route.router.sharding.validator.routingresult.RoutingResultValidator;
import org.apache.shardingsphere.core.route.type.RoutingResult;
import org.apache.shardingsphere.core.route.type.RoutingUnit;
import org.apache.shardingsphere.core.route.type.TableUnit;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.core.rule.TableRule;
import java.util.Collection;
import java.util.LinkedList;
/**
* Abstract routing result validator.
*
* @author sunbufu
* @author zhangliang
*/
@RequiredArgsConstructor
public abstract class AbstractRoutingResultValidator implements RoutingResultValidator {
@Getter
private final ShardingRule shardingRule;
private final ShardingSphereMetaData metaData;
@Override
public final void validate(final ShardingOptimizedStatement shardingStatement, final ShardingConditions shardingConditions, final RoutingResult routingResult) {
if (shardingStatement instanceof ShardingDropIndexOptimizedStatement) {
return;
}
Multimap<RoutingUnit, TableUnit> unconfiguredRoutingUnits = getUnconfiguredRoutingUnit(shardingStatement, routingResult.getRoutingUnits());
if (!unconfiguredRoutingUnits.isEmpty()) {
throwException(shardingStatement, shardingConditions, unconfiguredRoutingUnits);
}
}
private Multimap<RoutingUnit, TableUnit> getUnconfiguredRoutingUnit(final ShardingOptimizedStatement shardingStatement, final Collection<RoutingUnit> routingUnits) {
Multimap<RoutingUnit, TableUnit> result = HashMultimap.create();
for (RoutingUnit each : routingUnits) {
result.putAll(each, getUnconfiguredTableUnit(shardingStatement, each));
}
return result;
}
private Collection<TableUnit> getUnconfiguredTableUnit(final ShardingOptimizedStatement shardingStatement, final RoutingUnit routingUnit) {
Collection<TableUnit> result = new LinkedList<>();
for (TableUnit each : routingUnit.getTableUnits()) {
if (!containsInShardingRule(routingUnit.getDataSourceName(), each) && !containsInMetaData(shardingStatement, routingUnit.getDataSourceName(), each.getActualTableName())) {
result.add(each);
}
}
return result;
}
private boolean containsInShardingRule(final String dataSourceName, final TableUnit tableUnit) {
Optional<TableRule> tableRule = shardingRule.findTableRule(tableUnit.getLogicTableName());
return tableRule.isPresent() && tableRule.get().getActualTableNames(dataSourceName).contains(tableUnit.getActualTableName());
}
private boolean containsInMetaData(final ShardingOptimizedStatement shardingStatement, final String dataSourceName, final String actualTableName) {
if (shardingRule.getRuleConfiguration().getMasterSlaveRuleConfigs().isEmpty() && null == metaData.getDataSources().getDataSourceMetaData(dataSourceName)) {
return false;
}
if (shardingStatement.getSQLStatement() instanceof DMLStatement && !metaData.getTables().containsTable(actualTableName)) {
return false;
}
return true;
}
protected abstract void throwException(ShardingOptimizedStatement shardingStatement, ShardingConditions shardingConditions, Multimap<RoutingUnit, TableUnit> unconfiguredRoutingUnits);
}
/*
* 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.core.route.router.sharding.validator.routingresult.impl;
import com.google.common.collect.Multimap;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.metadata.ShardingSphereMetaData;
import org.apache.shardingsphere.core.optimize.sharding.statement.ShardingOptimizedStatement;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.core.route.type.RoutingUnit;
import org.apache.shardingsphere.core.route.type.TableUnit;
import org.apache.shardingsphere.core.rule.ShardingRule;
import java.util.ArrayList;
import java.util.Collection;
/**
* Complex routing result validator.
*
* @author sunbufu
* @author zhangliang
*/
public final class ComplexRoutingResultValidator extends AbstractRoutingResultValidator {
public ComplexRoutingResultValidator(final ShardingRule shardingRule, final ShardingSphereMetaData metaData) {
super(shardingRule, metaData);
}
@Override
protected void throwException(final ShardingOptimizedStatement shardingStatement, final ShardingConditions shardingConditions, final Multimap<RoutingUnit, TableUnit> unconfiguredRoutingUnits) {
RoutingUnit routingUnit = unconfiguredRoutingUnits.keySet().iterator().next();
Collection<String> unconfiguredDataNodes = new ArrayList<>(unconfiguredRoutingUnits.get(routingUnit).size());
for (TableUnit each : unconfiguredRoutingUnits.get(routingUnit)) {
unconfiguredDataNodes.add(routingUnit.getDataSourceName() + "." + each.getActualTableName());
}
if (!unconfiguredDataNodes.isEmpty()) {
throw new ShardingException(String.format("We get some absent DataNodes=%s in routing result, please check the configuration of rule and data node.", unconfiguredDataNodes));
}
}
}
/*
* 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.core.route.router.sharding.validator.routingresult.impl;
import com.google.common.base.Optional;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimap;
import org.apache.shardingsphere.api.config.sharding.TableRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.strategy.ShardingStrategyConfiguration;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.metadata.ShardingSphereMetaData;
import org.apache.shardingsphere.core.optimize.sharding.statement.ShardingOptimizedStatement;
import org.apache.shardingsphere.core.optimize.sharding.statement.dml.ShardingConditionOptimizedStatement;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.core.route.type.RoutingUnit;
import org.apache.shardingsphere.core.route.type.TableUnit;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.core.strategy.route.value.RouteValue;
import java.util.Collection;
import java.util.LinkedList;
/**
* Standard routing result validator.
*
* @author sunbufu
* @author zhangliang
*/
public final class StandardRoutingResultValidator extends AbstractRoutingResultValidator {
public StandardRoutingResultValidator(final ShardingRule shardingRule, final ShardingSphereMetaData metaData) {
super(shardingRule, metaData);
}
@Override
protected void throwException(final ShardingOptimizedStatement shardingStatement, final ShardingConditions shardingConditions, final Multimap<RoutingUnit, TableUnit> unconfiguredRoutingUnits) {
RoutingUnit routingUnit = unconfiguredRoutingUnits.keySet().iterator().next();
Collection<String> absentDataNodes = Lists.newArrayListWithExpectedSize(unconfiguredRoutingUnits.get(routingUnit).size());
ShardingStrategyConfiguration databaseStrategy = null;
ShardingStrategyConfiguration tableStrategy = null;
if (null != getShardingRule().getRuleConfiguration().getDefaultDatabaseShardingStrategyConfig()) {
databaseStrategy = getShardingRule().getRuleConfiguration().getDefaultDatabaseShardingStrategyConfig();
}
if (null != getShardingRule().getRuleConfiguration().getDefaultTableShardingStrategyConfig()) {
tableStrategy = getShardingRule().getRuleConfiguration().getDefaultTableShardingStrategyConfig();
}
for (TableUnit each : unconfiguredRoutingUnits.get(routingUnit)) {
absentDataNodes.add(routingUnit.getDataSourceName() + "." + each.getActualTableName());
Optional<TableRuleConfiguration> tableRuleConfiguration = getTableRuleConfiguration(getShardingRule().getRuleConfiguration().getTableRuleConfigs(), each.getLogicTableName());
if (tableRuleConfiguration.isPresent()) {
databaseStrategy = tableRuleConfiguration.get().getDatabaseShardingStrategyConfig();
tableStrategy = tableRuleConfiguration.get().getTableShardingStrategyConfig();
}
}
if (!absentDataNodes.isEmpty()) {
StringBuilder detail = new StringBuilder();
if (null != databaseStrategy) {
detail.append("DatabaseStrategy=[").append(databaseStrategy).append("], ");
}
detail.append("TableStrategy=[").append(tableStrategy).append("], ");
if (shardingStatement instanceof ShardingConditionOptimizedStatement) {
detail.append("with ").append(getAllRouteValues(shardingConditions.getConditions())).append(", ");
}
throwExceptionForAbsentDataNode(absentDataNodes, detail);
}
}
private void throwExceptionForAbsentDataNode(final Collection<String> absentDataNodes, final CharSequence detail) {
String msg = "We get some absent DataNodes=" + absentDataNodes + " in routing result, " + (Strings.isNullOrEmpty(detail.toString()) ? "" : detail)
+ "please check the configuration of rule and data node.";
throw new ShardingException(msg.replace("%", "%%"));
}
private Optional<TableRuleConfiguration> getTableRuleConfiguration(final Collection<TableRuleConfiguration> tableRuleConfigurations, final String tableName) {
for (TableRuleConfiguration each : tableRuleConfigurations) {
if (tableName.equals(each.getLogicTable())) {
return Optional.of(each);
}
}
return Optional.absent();
}
private Collection<RouteValue> getAllRouteValues(final Collection<ShardingCondition> shardingConditions) {
Collection<RouteValue> result = new LinkedList<>();
for (ShardingCondition each : shardingConditions) {
result.addAll(each.getRouteValues());
}
return result;
}
}
......@@ -194,12 +194,13 @@ public final class StandardRoutingEngine implements RoutingEngine {
}
private Collection<String> routeDataSources(final TableRule tableRule, final List<RouteValue> databaseShardingValues) {
Collection<String> availableTargetDatabases = tableRule.getActualDatasourceNames();
if (databaseShardingValues.isEmpty()) {
return availableTargetDatabases;
return tableRule.getActualDatasourceNames();
}
Collection<String> result = new LinkedHashSet<>(shardingRule.getDatabaseShardingStrategy(tableRule).doSharding(availableTargetDatabases, databaseShardingValues));
Collection<String> result = new LinkedHashSet<>(shardingRule.getDatabaseShardingStrategy(tableRule).doSharding(tableRule.getActualDatasourceNames(), databaseShardingValues));
Preconditions.checkState(!result.isEmpty(), "no database route info");
Preconditions.checkState(tableRule.getActualDatasourceNames().containsAll(result),
"Some routed data sources do not belong to configured data sources. routed data sources: `%s`, configured data sources: `%s`", result, tableRule.getActualDatasourceNames());
return result;
}
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.route.router.sharding.validator.statement.impl;
package org.apache.shardingsphere.core.route.router.sharding.validator.impl;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.parse.sql.segment.dml.column.ColumnSegment;
......
......@@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.shardingsphere.core.route.router.sharding.validator.statement.impl;
package org.apache.shardingsphere.core.route.router.sharding.validator.impl;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.parse.sql.segment.dml.assignment.AssignmentSegment;
......
/*
* 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.core.route.router.sharding.validator.routingresult;
import com.google.common.base.Optional;
import org.apache.shardingsphere.api.config.masterslave.MasterSlaveRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.ShardingRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.TableRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.strategy.InlineShardingStrategyConfiguration;
import org.apache.shardingsphere.api.config.sharding.strategy.ShardingStrategyConfiguration;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.metadata.ShardingSphereMetaData;
import org.apache.shardingsphere.core.metadata.datasource.DataSourceMetas;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.optimize.sharding.statement.ShardingOptimizedStatement;
import org.apache.shardingsphere.core.parse.sql.statement.dml.DMLStatement;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.core.route.router.sharding.validator.routingresult.impl.ComplexRoutingResultValidator;
import org.apache.shardingsphere.core.route.type.RoutingResult;
import org.apache.shardingsphere.core.route.type.RoutingUnit;
import org.apache.shardingsphere.core.route.type.TableUnit;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.core.rule.TableRule;
import org.apache.shardingsphere.spi.database.DataSourceMetaData;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertThat;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public final class ComplexRoutingResultValidatorTest {
@Test
public void assertValidateOthersRoutingResult() {
new ComplexRoutingResultValidator(
getShardingRule(), getMetaData()).validate(getShardingOptimizedStatement(), new ShardingConditions(Collections.<ShardingCondition>emptyList()), getRoutingResult());
}
@Test
public void assertValidateOthersRoutingResultWithAbsentDatabase() {
String msg = null;
try {
new ComplexRoutingResultValidator(getShardingRule(), getMetaData()).validate(
getShardingOptimizedStatement(), new ShardingConditions(Collections.<ShardingCondition>emptyList()), getRoutingResultWithAbsentDatabase());
} catch (ShardingException ex) {
msg = ex.getMessage();
}
assertThat(msg, is("We get some absent DataNodes=[db_2.t_order_0] in routing result, please check the configuration of rule and data node."));
}
private ShardingOptimizedStatement getShardingOptimizedStatement() {
ShardingOptimizedStatement result = mock(ShardingOptimizedStatement.class);
doReturn(mock(DMLStatement.class)).when(result).getSQLStatement();
return result;
}
private ShardingRule getShardingRule() {
ShardingRule result = mock(ShardingRule.class);
doAnswer(new Answer() {
@Override
public Object answer(final InvocationOnMock invocation) {
if ("t_order".equals(invocation.getArgument(0))) {
return Optional.of(getTableRule());
} else {
return Optional.absent();
}
}
}).when(result).findTableRule(anyString());
doReturn(getRuleConfiguration()).when(result).getRuleConfiguration();
return result;
}
private TableRule getTableRule() {
TableRule result = mock(TableRule.class);
when(result.getActualTableNames("db_0")).thenReturn(Arrays.asList("t_order_0", "t_order_1"));
when(result.getActualTableNames("db_1")).thenReturn(Arrays.asList("t_order_0", "t_order_1"));
return result;
}
private ShardingRuleConfiguration getRuleConfiguration() {
ShardingRuleConfiguration result = mock(ShardingRuleConfiguration.class);
doReturn(getDefaultShardingStrategyConfiguration()).when(result).getDefaultDatabaseShardingStrategyConfig();
doReturn(getDefaultShardingStrategyConfiguration()).when(result).getDefaultTableShardingStrategyConfig();
doReturn(Collections.singleton(getTableRuleConfiguration())).when(result).getTableRuleConfigs();
when(result.getMasterSlaveRuleConfigs()).thenReturn(Collections.<MasterSlaveRuleConfiguration>emptyList());
return result;
}
private ShardingStrategyConfiguration getDefaultShardingStrategyConfiguration() {
ShardingStrategyConfiguration result = mock(InlineShardingStrategyConfiguration.class);
when(result.toString()).thenReturn("Inline{shardingColumn='order_id', algorithmExpression='t_order_${order_id % 3}'}");
return result;
}
private TableRuleConfiguration getTableRuleConfiguration() {
TableRuleConfiguration result = mock(TableRuleConfiguration.class);
when(result.getLogicTable()).thenReturn("t_order");
doReturn(getShardingStrategyConfiguration()).when(result).getDatabaseShardingStrategyConfig();
doReturn(getShardingStrategyConfiguration()).when(result).getTableShardingStrategyConfig();
return result;
}
private ShardingStrategyConfiguration getShardingStrategyConfiguration() {
ShardingStrategyConfiguration result = mock(InlineShardingStrategyConfiguration.class);
when(result.toString()).thenReturn("Inline{shardingColumn='order_id', algorithmExpression='t_order_${order_id % 2}'}");
return result;
}
private RoutingResult getRoutingResultWithAbsentDatabase() {
RoutingResult result = mock(RoutingResult.class);
doReturn(getRoutingUnitsWithAbsentDatabase()).when(result).getRoutingUnits();
return result;
}
private Collection<RoutingUnit> getRoutingUnitsWithAbsentDatabase() {
RoutingUnit routingUnit = mock(RoutingUnit.class);
doReturn("db_2").when(routingUnit).getDataSourceName();
doReturn(Collections.singletonList(getTableUnit())).when(routingUnit).getTableUnits();
return Collections.singleton(routingUnit);
}
private ShardingSphereMetaData getMetaData() {
ShardingSphereMetaData result = mock(ShardingSphereMetaData.class);
doReturn(getDataSourceMetas()).when(result).getDataSources();
doReturn(getTableMetas()).when(result).getTables();
return result;
}
private TableMetas getTableMetas() {
TableMetas tables = mock(TableMetas.class);
when(tables.containsTable("t_order")).thenReturn(true);
when(tables.containsTable("t_order_0")).thenReturn(true);
when(tables.containsTable("t_order_1")).thenReturn(true);
return tables;
}
private DataSourceMetas getDataSourceMetas() {
DataSourceMetas dataSourceMetas = mock(DataSourceMetas.class);
DataSourceMetaData dataSourceMetaData0 = mock(DataSourceMetaData.class);
doReturn(dataSourceMetaData0).when(dataSourceMetas).getDataSourceMetaData("db_0");
return dataSourceMetas;
}
private RoutingResult getRoutingResult() {
RoutingResult result = mock(RoutingResult.class);
doReturn(getRoutingUnits()).when(result).getRoutingUnits();
return result;
}
private Collection<RoutingUnit> getRoutingUnits() {
RoutingUnit routingUnit = mock(RoutingUnit.class);
doReturn("db_0").when(routingUnit).getDataSourceName();
doReturn(Collections.singletonList(getTableUnit())).when(routingUnit).getTableUnits();
return Collections.singleton(routingUnit);
}
private TableUnit getTableUnit() {
TableUnit result = mock(TableUnit.class);
when(result.getActualTableName()).thenReturn("t_order_0");
when(result.getLogicTableName()).thenReturn("t_order");
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.core.route.router.sharding.validator.routingresult;
import com.google.common.base.Optional;
import org.apache.shardingsphere.api.config.masterslave.MasterSlaveRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.ShardingRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.TableRuleConfiguration;
import org.apache.shardingsphere.api.config.sharding.strategy.InlineShardingStrategyConfiguration;
import org.apache.shardingsphere.api.config.sharding.strategy.ShardingStrategyConfiguration;
import org.apache.shardingsphere.core.exception.ShardingException;
import org.apache.shardingsphere.core.metadata.ShardingSphereMetaData;
import org.apache.shardingsphere.core.metadata.datasource.DataSourceMetas;
import org.apache.shardingsphere.core.metadata.table.TableMetas;
import org.apache.shardingsphere.core.optimize.sharding.statement.ShardingOptimizedStatement;
import org.apache.shardingsphere.core.optimize.sharding.statement.ddl.ShardingDropIndexOptimizedStatement;
import org.apache.shardingsphere.core.optimize.sharding.statement.dml.ShardingConditionOptimizedStatement;
import org.apache.shardingsphere.core.parse.sql.statement.dml.DMLStatement;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingCondition;
import org.apache.shardingsphere.core.route.router.sharding.condition.ShardingConditions;
import org.apache.shardingsphere.core.route.router.sharding.validator.routingresult.impl.StandardRoutingResultValidator;
import org.apache.shardingsphere.core.route.type.RoutingResult;
import org.apache.shardingsphere.core.route.type.RoutingUnit;
import org.apache.shardingsphere.core.route.type.TableUnit;
import org.apache.shardingsphere.core.rule.ShardingRule;
import org.apache.shardingsphere.core.rule.TableRule;
import org.apache.shardingsphere.core.strategy.route.value.ListRouteValue;
import org.apache.shardingsphere.core.strategy.route.value.RouteValue;
import org.apache.shardingsphere.spi.database.DataSourceMetaData;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertThat;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public final class StandardRoutingResultValidatorTest {
@Test
public void assertValidateWithShardingDropIndexOptimizedStatement() {
new StandardRoutingResultValidator(getShardingRule(), getMetaData()).validate(
mock(ShardingDropIndexOptimizedStatement.class), new ShardingConditions(Collections.<ShardingCondition>emptyList()), getRoutingResult());
}
@Test
public void assertValidateStandardRoutingResult() {
new StandardRoutingResultValidator(getShardingRule(), getMetaData()).validate(
getShardingOptimizedStatement(), new ShardingConditions(Collections.<ShardingCondition>emptyList()), getRoutingResult());
}
@Test
public void assertValidateStandardRoutingResultWithAbsentDatabaseAndDefaultStrategy() {
String msg = null;
try {
new StandardRoutingResultValidator(getShardingRuleWithDefaultStrategy(), getMetaData()).validate(
getShardingOptimizedStatement(), new ShardingConditions(Collections.<ShardingCondition>emptyList()), getRoutingResultWithAbsentDatabase());
} catch (ShardingException ex) {
msg = ex.getMessage();
}
assertThat(msg, is("We get some absent DataNodes=[db_2.t_order_0] in routing result, DatabaseStrategy=[Inline{shardingColumn='order_id', "
+ "algorithmExpression='t_order_${order_id % 3}'}], TableStrategy=[Inline{shardingColumn='order_id', algorithmExpression='t_order_${order_id % 3}'}], "
+ "please check the configuration of rule and data node."));
}
private ShardingRule getShardingRuleWithDefaultStrategy() {
ShardingRule result = mock(ShardingRule.class);
doAnswer(new Answer() {
@Override
public Object answer(final InvocationOnMock invocation) {
if ("t_order".equals(invocation.getArgument(0))) {
return Optional.of(getTableRule());
} else {
return Optional.absent();
}
}
}).when(result).findTableRule(anyString());
doReturn(getRuleConfigurationWithDefaultStrategy()).when(result).getRuleConfiguration();
return result;
}
private ShardingRuleConfiguration getRuleConfigurationWithDefaultStrategy() {
ShardingRuleConfiguration result = mock(ShardingRuleConfiguration.class);
doReturn(getDefaultShardingStrategyConfiguration()).when(result).getDefaultDatabaseShardingStrategyConfig();
doReturn(getDefaultShardingStrategyConfiguration()).when(result).getDefaultTableShardingStrategyConfig();
when(result.getMasterSlaveRuleConfigs()).thenReturn(Collections.<MasterSlaveRuleConfiguration>emptyList());
return result;
}
@Test
public void assertValidateStandardRoutingResultWithAbsentDatabase() {
String message = null;
try {
new StandardRoutingResultValidator(getShardingRule(), getMetaData()).validate(
getShardingOptimizedStatement(), new ShardingConditions(Collections.<ShardingCondition>emptyList()), getRoutingResultWithAbsentDatabase());
} catch (ShardingException ex) {
message = ex.getMessage();
}
assertThat(message, is("We get some absent DataNodes=[db_2.t_order_0] in routing result, DatabaseStrategy=[Inline{shardingColumn='order_id', "
+ "algorithmExpression='t_order_${order_id % 2}'}], TableStrategy=[Inline{shardingColumn='order_id', algorithmExpression='t_order_${order_id % 2}'}], "
+ "please check the configuration of rule and data node."));
}
private ShardingOptimizedStatement getShardingOptimizedStatement() {
ShardingOptimizedStatement result = mock(ShardingOptimizedStatement.class);
doReturn(mock(DMLStatement.class)).when(result).getSQLStatement();
return result;
}
@Test
public void assertValidateStandardRoutingResultWithAbsentDatabaseAndRouteValues() {
String message = null;
try {
new StandardRoutingResultValidator(getShardingRule(), getMetaData()).validate(getShardingConditionOptimizedStatement(), getShardingConditions(), getRoutingResultWithAbsentDatabase());
} catch (ShardingException ex) {
message = ex.getMessage();
}
assertThat(message, is("We get some absent DataNodes=[db_2.t_order_0] in routing result, DatabaseStrategy=[Inline{shardingColumn='order_id', "
+ "algorithmExpression='t_order_${order_id % 2}'}], TableStrategy=[Inline{shardingColumn='order_id', algorithmExpression='t_order_${order_id % 2}'}], with [t_order_1.order_id = 1], "
+ "please check the configuration of rule and data node."));
}
private ShardingOptimizedStatement getShardingConditionOptimizedStatement() {
ShardingConditionOptimizedStatement result = mock(ShardingConditionOptimizedStatement.class);
doReturn(mock(DMLStatement.class)).when(result).getSQLStatement();
return result;
}
private ShardingConditions getShardingConditions() {
ShardingCondition shardingCondition = new ShardingCondition();
shardingCondition.getRouteValues().add(getListRouteValue());
ShardingConditions result = mock(ShardingConditions.class);
when(result.getConditions()).thenReturn(Collections.singletonList(shardingCondition));
return result;
}
private RouteValue getListRouteValue() {
return new ListRouteValue<>("order_id", "t_order_1", Collections.singleton(1));
}
private ShardingRule getShardingRule() {
ShardingRule result = mock(ShardingRule.class);
doAnswer(new Answer() {
@Override
public Object answer(final InvocationOnMock invocation) {
if ("t_order".equals(invocation.getArgument(0))) {
return Optional.of(getTableRule());
} else {
return Optional.absent();
}
}
}).when(result).findTableRule(anyString());
doReturn(getRuleConfiguration()).when(result).getRuleConfiguration();
return result;
}
private TableRule getTableRule() {
TableRule result = mock(TableRule.class);
when(result.getActualTableNames("db_0")).thenReturn(Arrays.asList("t_order_0", "t_order_1"));
when(result.getActualTableNames("db_1")).thenReturn(Arrays.asList("t_order_0", "t_order_1"));
return result;
}
private ShardingRuleConfiguration getRuleConfiguration() {
ShardingRuleConfiguration result = mock(ShardingRuleConfiguration.class);
doReturn(getDefaultShardingStrategyConfiguration()).when(result).getDefaultDatabaseShardingStrategyConfig();
doReturn(getDefaultShardingStrategyConfiguration()).when(result).getDefaultTableShardingStrategyConfig();
doReturn(Collections.singleton(getTableRuleConfiguration())).when(result).getTableRuleConfigs();
when(result.getMasterSlaveRuleConfigs()).thenReturn(Collections.<MasterSlaveRuleConfiguration>emptyList());
return result;
}
private ShardingStrategyConfiguration getDefaultShardingStrategyConfiguration() {
ShardingStrategyConfiguration result = mock(InlineShardingStrategyConfiguration.class);
when(result.toString()).thenReturn("Inline{shardingColumn='order_id', algorithmExpression='t_order_${order_id % 3}'}");
return result;
}
private TableRuleConfiguration getTableRuleConfiguration() {
TableRuleConfiguration result = mock(TableRuleConfiguration.class);
when(result.getLogicTable()).thenReturn("t_order");
doReturn(getShardingStrategyConfiguration()).when(result).getDatabaseShardingStrategyConfig();
doReturn(getShardingStrategyConfiguration()).when(result).getTableShardingStrategyConfig();
return result;
}
private ShardingStrategyConfiguration getShardingStrategyConfiguration() {
ShardingStrategyConfiguration result = mock(InlineShardingStrategyConfiguration.class);
when(result.toString()).thenReturn("Inline{shardingColumn='order_id', algorithmExpression='t_order_${order_id % 2}'}");
return result;
}
private RoutingResult getRoutingResultWithAbsentDatabase() {
RoutingResult result = mock(RoutingResult.class);
doReturn(getRoutingUnitsWithAbsentDatabase()).when(result).getRoutingUnits();
return result;
}
private Collection<RoutingUnit> getRoutingUnitsWithAbsentDatabase() {
RoutingUnit routingUnit = mock(RoutingUnit.class);
doReturn("db_2").when(routingUnit).getDataSourceName();
doReturn(Collections.singletonList(getTableUnit())).when(routingUnit).getTableUnits();
return Collections.singleton(routingUnit);
}
private ShardingSphereMetaData getMetaData() {
ShardingSphereMetaData result = mock(ShardingSphereMetaData.class);
doReturn(getDataSourceMetas()).when(result).getDataSources();
doReturn(getTableMetas()).when(result).getTables();
return result;
}
private TableMetas getTableMetas() {
TableMetas tables = mock(TableMetas.class);
when(tables.containsTable("t_order")).thenReturn(true);
when(tables.containsTable("t_order_0")).thenReturn(true);
when(tables.containsTable("t_order_1")).thenReturn(true);
return tables;
}
private DataSourceMetas getDataSourceMetas() {
DataSourceMetas dataSourceMetas = mock(DataSourceMetas.class);
DataSourceMetaData dataSourceMetaData0 = mock(DataSourceMetaData.class);
doReturn(dataSourceMetaData0).when(dataSourceMetas).getDataSourceMetaData("db_0");
return dataSourceMetas;
}
private RoutingResult getRoutingResult() {
RoutingResult result = mock(RoutingResult.class);
doReturn(getRoutingUnits()).when(result).getRoutingUnits();
return result;
}
private Collection<RoutingUnit> getRoutingUnits() {
RoutingUnit routingUnit = mock(RoutingUnit.class);
doReturn("db_0").when(routingUnit).getDataSourceName();
doReturn(Collections.singletonList(getTableUnit())).when(routingUnit).getTableUnits();
return Collections.singleton(routingUnit);
}
private TableUnit getTableUnit() {
TableUnit result = mock(TableUnit.class);
when(result.getActualTableName()).thenReturn("t_order_0");
when(result.getLogicTableName()).thenReturn("t_order");
return result;
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册