mirror of
https://github.com/actiontech/dble.git
synced 2026-01-07 21:30:22 -06:00
@@ -942,10 +942,10 @@ public final class DbleServer {
|
||||
public void run() {
|
||||
Map<String, UserStat> statMap = UserStatAnalyzer.getInstance().getUserStatMap();
|
||||
for (UserStat userStat : statMap.values()) {
|
||||
userStat.getSqlLastStat().recycle();
|
||||
userStat.getSqlRecorder().recycle();
|
||||
//userStat.getSqlLastStat().recycle();
|
||||
//userStat.getSqlRecorder().recycle();
|
||||
userStat.getSqlHigh().recycle();
|
||||
userStat.getSqlLargeRowStat().recycle();
|
||||
//userStat.getSqlLargeRowStat().recycle();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
@@ -63,6 +63,8 @@ public abstract class PhysicalDatasource {
|
||||
|
||||
private volatile boolean testConnSuccess = false;
|
||||
|
||||
private String dsVersion;
|
||||
|
||||
public PhysicalDatasource(DBHostConfig config, DataHostConfig hostConfig, boolean isReadNode) {
|
||||
this.size = config.getMaxCon();
|
||||
this.config = config;
|
||||
@@ -125,6 +127,14 @@ public abstract class PhysicalDatasource {
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public String getDsVersion() {
|
||||
return dsVersion;
|
||||
}
|
||||
|
||||
public void setDsVersion(String dsVersion) {
|
||||
this.dsVersion = dsVersion;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
@@ -483,7 +493,7 @@ public abstract class PhysicalDatasource {
|
||||
/**
|
||||
* used for init or reload
|
||||
*/
|
||||
public abstract boolean testConnection(String schema) throws IOException;
|
||||
public abstract boolean testConnection() throws IOException;
|
||||
|
||||
public long getHeartbeatRecoveryTime() {
|
||||
return heartbeatRecoveryTime;
|
||||
|
||||
@@ -0,0 +1,26 @@
|
||||
/*
|
||||
* Copyright (C) 2016-2020 ActionTech.
|
||||
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
|
||||
*/
|
||||
|
||||
package com.actiontech.dble.backend.mysql;
|
||||
|
||||
public final class VersionUtil {
|
||||
private VersionUtil() {
|
||||
}
|
||||
|
||||
public static final String TX_READ_ONLY = "tx_read_only";
|
||||
public static final String TRANSACTION_READ_ONLY = "transaction_read_only";
|
||||
public static final String TX_ISOLATION = "tx_isolation";
|
||||
public static final String TRANSACTION_ISOLATION = "transaction_isolation";
|
||||
|
||||
public static String getIsolationNameByVersion(String version) {
|
||||
if (version == null) {
|
||||
return null;
|
||||
} else if (version.startsWith("8")) {
|
||||
return TRANSACTION_ISOLATION;
|
||||
} else {
|
||||
return TX_ISOLATION;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -27,8 +27,6 @@ import java.nio.charset.StandardCharsets;
|
||||
public class MySQLDataSource extends PhysicalDatasource {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(MySQLDataSource.class);
|
||||
|
||||
|
||||
private final MySQLConnectionFactory factory;
|
||||
|
||||
public MySQLDataSource(DBHostConfig config, DataHostConfig hostConfig,
|
||||
@@ -80,7 +78,7 @@ public class MySQLDataSource extends PhysicalDatasource {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean testConnection(String schema) throws IOException {
|
||||
public boolean testConnection() throws IOException {
|
||||
|
||||
boolean isConnected = true;
|
||||
Socket socket = null;
|
||||
@@ -112,12 +110,13 @@ public class MySQLDataSource extends PhysicalDatasource {
|
||||
handshake.read(bin1);
|
||||
|
||||
String authPluginName = new String(handshake.getAuthPluginName());
|
||||
this.setDsVersion(new String(handshake.getServerVersion()));
|
||||
byte[] authPluginData = null;
|
||||
if (authPluginName.equals(new String(HandshakeV10Packet.NATIVE_PASSWORD_PLUGIN))) {
|
||||
/**
|
||||
* Phase 2: client to MySQL. Send auth packet.
|
||||
*/
|
||||
startAuthPacket(out, handshake, PasswordAuthPlugin.passwd(this.getConfig().getPassword(), handshake), schema, authPluginName);
|
||||
startAuthPacket(out, handshake, PasswordAuthPlugin.passwd(this.getConfig().getPassword(), handshake), authPluginName);
|
||||
/**
|
||||
* Phase 3: MySQL to client. send OK/ERROR packet.
|
||||
*/
|
||||
@@ -165,7 +164,7 @@ public class MySQLDataSource extends PhysicalDatasource {
|
||||
* Phase 2: client to MySQL. Send auth packet.
|
||||
*/
|
||||
try {
|
||||
startAuthPacket(out, handshake, PasswordAuthPlugin.passwdSha256(this.getConfig().getPassword(), handshake), schema, authPluginName);
|
||||
startAuthPacket(out, handshake, PasswordAuthPlugin.passwdSha256(this.getConfig().getPassword(), handshake), authPluginName);
|
||||
|
||||
BinaryPacket bin2 = new BinaryPacket();
|
||||
bin2.read(in);
|
||||
@@ -207,7 +206,7 @@ public class MySQLDataSource extends PhysicalDatasource {
|
||||
break;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOGGER.warn("connect the schema:" + schema + " failed");
|
||||
LOGGER.warn("testConnection failed");
|
||||
isConnected = false;
|
||||
}
|
||||
} else {
|
||||
@@ -238,16 +237,16 @@ public class MySQLDataSource extends PhysicalDatasource {
|
||||
}
|
||||
|
||||
|
||||
public void startAuthPacket(OutputStream out, HandshakeV10Packet handshake, byte[] passwordSented, String schema, String authPluginName) {
|
||||
public void startAuthPacket(OutputStream out, HandshakeV10Packet handshake, byte[] passwordSented, String authPluginName) {
|
||||
AuthPacket authPacket = new AuthPacket();
|
||||
authPacket.setPacketId(1);
|
||||
authPacket.setClientFlags(getClientFlagSha(schema != null));
|
||||
authPacket.setClientFlags(getClientFlagSha(false));
|
||||
authPacket.setMaxPacketSize(1024 * 1024 * 16);
|
||||
authPacket.setCharsetIndex(handshake.getServerCharsetIndex() & 0xff);
|
||||
authPacket.setUser(this.getConfig().getUser());
|
||||
try {
|
||||
authPacket.setPassword(passwordSented);
|
||||
authPacket.setDatabase(schema);
|
||||
authPacket.setDatabase(null);
|
||||
authPacket.setAuthPlugin(authPluginName);
|
||||
authPacket.writeWithKey(out);
|
||||
out.flush();
|
||||
|
||||
@@ -54,21 +54,21 @@ public abstract class BaseHandlerBuilder {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(BaseHandlerBuilder.class);
|
||||
private static AtomicLong sequenceId = new AtomicLong(0);
|
||||
protected NonBlockingSession session;
|
||||
protected HandlerBuilder hBuilder;
|
||||
HandlerBuilder hBuilder;
|
||||
protected DMLResponseHandler start;
|
||||
/* the current last handler */
|
||||
protected DMLResponseHandler currentLast;
|
||||
private DMLResponseHandler currentLast;
|
||||
private PlanNode node;
|
||||
protected Map<String, SchemaConfig> schemaConfigMap = new HashMap<>();
|
||||
Map<String, SchemaConfig> schemaConfigMap = new HashMap<>();
|
||||
/* the children can be push down */
|
||||
protected boolean canPushDown = false;
|
||||
boolean canPushDown = false;
|
||||
/* need common handler? like group by,order by,limit and so on */
|
||||
protected boolean needCommon = true;
|
||||
boolean needCommon = true;
|
||||
/* has where handler */
|
||||
protected boolean needWhereHandler = true;
|
||||
boolean needWhereHandler = true;
|
||||
|
||||
protected boolean isExplain = false;
|
||||
protected List<BaseHandlerBuilder> subQueryBuilderList = new CopyOnWriteArrayList<>();
|
||||
private List<BaseHandlerBuilder> subQueryBuilderList = new CopyOnWriteArrayList<>();
|
||||
|
||||
protected BaseHandlerBuilder(NonBlockingSession session, PlanNode node, HandlerBuilder hBuilder, boolean isExplain) {
|
||||
this.session = session;
|
||||
@@ -165,6 +165,10 @@ public abstract class BaseHandlerBuilder {
|
||||
GlobalVisitor visitor = new GlobalVisitor(node, true);
|
||||
visitor.visit();
|
||||
String sql = visitor.getSql().toString();
|
||||
Map<String, String> mapTableToSimple = visitor.getMapTableToSimple();
|
||||
for (Map.Entry<String, String> tableToSimple : mapTableToSimple.entrySet()) {
|
||||
sql = sql.replace(tableToSimple.getKey(), tableToSimple.getValue());
|
||||
}
|
||||
String randomDataNode = getRandomNode(node.getNoshardNode());
|
||||
RouteResultsetNode rrsNode = new RouteResultsetNode(randomDataNode, ServerParse.SELECT, sql);
|
||||
RouteResultsetNode[] rrss = new RouteResultsetNode[]{rrsNode};
|
||||
@@ -194,7 +198,7 @@ public abstract class BaseHandlerBuilder {
|
||||
/**
|
||||
* build common properties,like where,groupby,having,orderby,limit, and sendMakHandler(rename)
|
||||
*/
|
||||
protected void buildCommon() {
|
||||
private void buildCommon() {
|
||||
if (node.getWhereFilter() != null && needWhereHandler) {
|
||||
WhereHandler wh = new WhereHandler(getSequenceId(), session, node.getWhereFilter());
|
||||
addHandler(wh);
|
||||
@@ -277,7 +281,7 @@ public abstract class BaseHandlerBuilder {
|
||||
/**
|
||||
* add a handler into handler chain
|
||||
*/
|
||||
protected void addHandler(DMLResponseHandler bh) {
|
||||
void addHandler(DMLResponseHandler bh) {
|
||||
if (currentLast == null) {
|
||||
start = bh;
|
||||
currentLast = bh;
|
||||
@@ -296,9 +300,6 @@ public abstract class BaseHandlerBuilder {
|
||||
/**
|
||||
* if the node's parent handler has been ordered,it is no need to order again
|
||||
*
|
||||
* @param planNode
|
||||
* @param orderBys
|
||||
* @return
|
||||
*/
|
||||
private boolean isOrderNeeded(PlanNode planNode, List<Order> orderBys) {
|
||||
if (planNode instanceof TableNode || PlanUtil.isGlobalOrER(planNode))
|
||||
@@ -314,9 +315,6 @@ public abstract class BaseHandlerBuilder {
|
||||
/**
|
||||
* the order way of join node stored in left join on orders and right join on orders
|
||||
*
|
||||
* @param jn
|
||||
* @param orderBys
|
||||
* @return
|
||||
*/
|
||||
private boolean isJoinNodeOrderMatch(JoinNode jn, List<Order> orderBys) {
|
||||
// onCondition column in orderBys will be saved to onOrders,
|
||||
@@ -341,11 +339,6 @@ public abstract class BaseHandlerBuilder {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param qn
|
||||
* @param orderBys
|
||||
* @return
|
||||
*/
|
||||
private boolean isQueryNodeOrderMatch(QueryNode qn, List<Order> orderBys) {
|
||||
List<Order> childOrders = qn.getChild().getOrderBys();
|
||||
List<Order> pushedOrders = PlanUtil.getPushDownOrders(qn, orderBys);
|
||||
@@ -355,9 +348,6 @@ public abstract class BaseHandlerBuilder {
|
||||
/**
|
||||
* try to merger the order of 'order by' syntax to columnsSelected
|
||||
*
|
||||
* @param columnsSelected
|
||||
* @param orderBys
|
||||
* @return
|
||||
*/
|
||||
private List<Order> mergeOrderBy(List<Item> columnsSelected, List<Order> orderBys) {
|
||||
List<Integer> orderIndexes = new ArrayList<>();
|
||||
@@ -381,7 +371,7 @@ public abstract class BaseHandlerBuilder {
|
||||
return newOrderByList;
|
||||
}
|
||||
|
||||
protected static boolean nodeHasGroupBy(PlanNode arg) {
|
||||
private static boolean nodeHasGroupBy(PlanNode arg) {
|
||||
return (arg.getSumFuncs().size() > 0 || arg.getGroupBys().size() > 0);
|
||||
}
|
||||
|
||||
@@ -389,7 +379,7 @@ public abstract class BaseHandlerBuilder {
|
||||
return sequenceId.incrementAndGet();
|
||||
}
|
||||
|
||||
protected void buildMergeHandler(PlanNode planNode, RouteResultsetNode[] rrssArray) {
|
||||
void buildMergeHandler(PlanNode planNode, RouteResultsetNode[] rrssArray) {
|
||||
hBuilder.checkRRSs(rrssArray);
|
||||
List<Order> orderBys = planNode.getGroupBys().size() > 0 ? planNode.getGroupBys() : planNode.getOrderBys();
|
||||
boolean isEasyMerge = rrssArray.length == 1 || (orderBys == null || orderBys.size() == 0);
|
||||
@@ -403,7 +393,7 @@ public abstract class BaseHandlerBuilder {
|
||||
addHandler(mh);
|
||||
}
|
||||
|
||||
protected String getRandomNode(Set<String> dataNodes) {
|
||||
String getRandomNode(Set<String> dataNodes) {
|
||||
String randomDatenode = null;
|
||||
int index = (int) (System.currentTimeMillis() % dataNodes.size());
|
||||
int i = 0;
|
||||
@@ -417,14 +407,14 @@ public abstract class BaseHandlerBuilder {
|
||||
return randomDatenode;
|
||||
}
|
||||
|
||||
protected TableConfig getTableConfig(String schema, String table) {
|
||||
TableConfig getTableConfig(String schema, String table) {
|
||||
SchemaConfig schemaConfig = schemaConfigMap.get(schema);
|
||||
if (schemaConfig == null)
|
||||
return null;
|
||||
return schemaConfig.getTables().get(table);
|
||||
}
|
||||
|
||||
protected void handleBlockingSubQuery() {
|
||||
void handleBlockingSubQuery() {
|
||||
if (node.getSubQueries().size() == 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
@@ -13,6 +13,7 @@ import com.actiontech.dble.backend.mysql.nio.handler.query.impl.join.JoinHandler
|
||||
import com.actiontech.dble.backend.mysql.nio.handler.query.impl.join.NotInHandler;
|
||||
import com.actiontech.dble.backend.mysql.nio.handler.util.CallBackHandler;
|
||||
import com.actiontech.dble.config.ErrorCode;
|
||||
import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.plan.common.exception.MySQLOutPutException;
|
||||
import com.actiontech.dble.plan.common.item.Item;
|
||||
import com.actiontech.dble.plan.common.item.Item.ItemType;
|
||||
@@ -22,19 +23,17 @@ import com.actiontech.dble.plan.common.item.function.operator.cmpfunc.ItemFuncIn
|
||||
import com.actiontech.dble.plan.node.JoinNode;
|
||||
import com.actiontech.dble.plan.node.PlanNode;
|
||||
import com.actiontech.dble.plan.util.PlanUtil;
|
||||
import com.actiontech.dble.route.RouteResultsetNode;
|
||||
import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.server.NonBlockingSession;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.*;
|
||||
|
||||
import static com.actiontech.dble.plan.optimizer.JoinStrategyProcessor.NEED_REPLACE;
|
||||
|
||||
class JoinNodeHandlerBuilder extends BaseHandlerBuilder {
|
||||
private JoinNode node;
|
||||
|
||||
protected JoinNodeHandlerBuilder(NonBlockingSession session, JoinNode node, HandlerBuilder hBuilder, boolean isExplain) {
|
||||
JoinNodeHandlerBuilder(NonBlockingSession session, JoinNode node, HandlerBuilder hBuilder, boolean isExplain) {
|
||||
super(session, node, hBuilder, isExplain);
|
||||
this.node = node;
|
||||
}
|
||||
@@ -57,19 +56,28 @@ class JoinNodeHandlerBuilder extends BaseHandlerBuilder {
|
||||
PushDownVisitor pdVisitor = new PushDownVisitor(node, true);
|
||||
MergeBuilder mergeBuilder = new MergeBuilder(session, node, needCommon, pdVisitor);
|
||||
String sql = null;
|
||||
Map<String, String> mapTableToSimple = new HashMap<>();
|
||||
if (node.getAst() != null && node.getParent() == null) { // it's root
|
||||
pdVisitor.visit();
|
||||
sql = pdVisitor.getSql().toString();
|
||||
mapTableToSimple = pdVisitor.getMapTableToSimple();
|
||||
}
|
||||
RouteResultsetNode[] rrssArray;
|
||||
SchemaConfig schemaConfig;
|
||||
String schemaName = this.session.getSource().getSchema();
|
||||
if (schemaName != null) {
|
||||
schemaConfig = schemaConfigMap.get(schemaName);
|
||||
} else {
|
||||
schemaConfig = schemaConfigMap.entrySet().iterator().next().getValue(); //random schemaConfig
|
||||
}
|
||||
RouteResultset rrs;
|
||||
// maybe some node is view
|
||||
if (sql == null) {
|
||||
rrssArray = mergeBuilder.construct().getNodes();
|
||||
rrs = mergeBuilder.construct(schemaConfig);
|
||||
} else {
|
||||
rrssArray = mergeBuilder.constructByStatement(sql, node.getAst()).getNodes();
|
||||
rrs = mergeBuilder.constructByStatement(sql, mapTableToSimple, node.getAst(), schemaConfig);
|
||||
}
|
||||
this.needCommon = mergeBuilder.getNeedCommonFlag();
|
||||
buildMergeHandler(node, rrssArray);
|
||||
buildMergeHandler(node, rrs.getNodes());
|
||||
} catch (Exception e) {
|
||||
throw new MySQLOutPutException(ErrorCode.ER_QUERYHANDLER, "", "join node mergebuild exception! Error:" + e.getMessage(), e);
|
||||
}
|
||||
@@ -185,11 +193,8 @@ class JoinNodeHandlerBuilder extends BaseHandlerBuilder {
|
||||
/**
|
||||
* generate filter for big table according to tmp(small) table's result
|
||||
*
|
||||
* @param tnBig
|
||||
* @param keyToPass
|
||||
* @param valueSet
|
||||
*/
|
||||
protected void buildNestFilters(PlanNode tnBig, Item keyToPass, Set<String> valueSet, int maxPartSize) {
|
||||
private void buildNestFilters(PlanNode tnBig, Item keyToPass, Set<String> valueSet, int maxPartSize) {
|
||||
List<Item> strategyFilters = tnBig.getNestLoopFilters();
|
||||
List<Item> partList = null;
|
||||
Item keyInBig = PlanUtil.pushDownItem(node, keyToPass);
|
||||
@@ -197,9 +202,8 @@ class JoinNodeHandlerBuilder extends BaseHandlerBuilder {
|
||||
for (String value : valueSet) {
|
||||
if (partList == null)
|
||||
partList = new ArrayList<>();
|
||||
if (value == null) { // is null will never join
|
||||
continue;
|
||||
} else {
|
||||
if (value != null) {
|
||||
// is null will never join
|
||||
partList.add(new ItemString(value));
|
||||
if (++partSize >= maxPartSize) {
|
||||
List<Item> argList = new ArrayList<>();
|
||||
|
||||
@@ -14,6 +14,7 @@ import com.actiontech.dble.plan.node.TableNode;
|
||||
import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.druid.impl.DruidSingleUnitSelectParser;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.actiontech.dble.server.NonBlockingSession;
|
||||
import com.actiontech.dble.server.parser.ServerParse;
|
||||
@@ -29,7 +30,7 @@ public class MergeBuilder {
|
||||
private boolean needCommonFlag;
|
||||
private PlanNode node;
|
||||
private NonBlockingSession session;
|
||||
protected Map<String, SchemaConfig> schemaConfigMap = new HashMap<>();
|
||||
private Map<String, SchemaConfig> schemaConfigMap = new HashMap<>();
|
||||
private PushDownVisitor pdVisitor;
|
||||
|
||||
MergeBuilder(NonBlockingSession session, PlanNode node, boolean needCommon, PushDownVisitor pdVisitor) {
|
||||
@@ -46,33 +47,34 @@ public class MergeBuilder {
|
||||
* @return RouteResultset
|
||||
* @throws SQLException SQLException
|
||||
*/
|
||||
public RouteResultset construct() throws SQLException {
|
||||
public RouteResultset construct(SchemaConfig schemaConfig) throws SQLException {
|
||||
pdVisitor.visit();
|
||||
String sql = pdVisitor.getSql().toString();
|
||||
return constructByQuery(sql);
|
||||
return constructByQuery(pdVisitor.getSql().toString(), pdVisitor.getMapTableToSimple(), schemaConfig);
|
||||
}
|
||||
|
||||
public RouteResultset constructByQuery(String sql) throws SQLException {
|
||||
private RouteResultset constructByQuery(String sql, Map<String, String> mapTableToSimple, SchemaConfig schemaConfig) throws SQLException {
|
||||
SQLStatementParser parser = new MySqlStatementParser(sql);
|
||||
SQLSelectStatement select = (SQLSelectStatement) parser.parseStatement();
|
||||
return constructByStatement(sql, select);
|
||||
return constructByStatement(sql, mapTableToSimple, select, schemaConfig);
|
||||
}
|
||||
|
||||
public RouteResultset constructByStatement(String sql, SQLSelectStatement select) throws SQLException {
|
||||
ServerSchemaStatVisitor visitor = new ServerSchemaStatVisitor();
|
||||
DruidSingleUnitSelectParser druidParser = new DruidSingleUnitSelectParser();
|
||||
|
||||
RouteResultset constructByStatement(String sql, Map<String, String> mapTableToSimple, SQLSelectStatement select, SchemaConfig schemaConfig) throws SQLException {
|
||||
RouteResultset rrs = new RouteResultset(sql, ServerParse.SELECT);
|
||||
String pushDownSQL = rrs.getStatement();
|
||||
for (Map.Entry<String, String> tableToSimple : mapTableToSimple.entrySet()) {
|
||||
pushDownSQL = pushDownSQL.replace(tableToSimple.getKey(), tableToSimple.getValue());
|
||||
}
|
||||
rrs.setStatement(pushDownSQL);
|
||||
LayerCachePool pool = DbleServer.getInstance().getRouterService().getTableId2DataNodeCache();
|
||||
Map<String, SchemaConfig> tableConfigMap = new HashMap<>();
|
||||
Map<Pair<String, String>, SchemaConfig> tableConfigMap = new HashMap<>();
|
||||
for (TableNode tn : node.getReferedTableNodes()) {
|
||||
if (schemaConfigMap.get(tn.getSchema()) != null) {
|
||||
tableConfigMap.put(tn.getTableName(), schemaConfigMap.get(tn.getSchema()));
|
||||
tableConfigMap.put(new Pair<>(tn.getSchema(), tn.getTableName()), schemaConfigMap.get(tn.getSchema()));
|
||||
}
|
||||
}
|
||||
DruidSingleUnitSelectParser druidParser = new DruidSingleUnitSelectParser();
|
||||
druidParser.setSchemaMap(tableConfigMap);
|
||||
return RouterUtil.routeFromParserComplex(druidParser, tableConfigMap, rrs, select, sql, pool, visitor, session.getSource(), node);
|
||||
|
||||
return RouterUtil.routeFromParserComplex(schemaConfig, druidParser, tableConfigMap, rrs, select, sql, pool, new ServerSchemaStatVisitor(), session.getSource(), node);
|
||||
}
|
||||
|
||||
/* -------------------- getter/setter -------------------- */
|
||||
|
||||
@@ -5,26 +5,27 @@
|
||||
|
||||
package com.actiontech.dble.backend.mysql.nio.handler.builder;
|
||||
|
||||
import com.actiontech.dble.DbleServer;
|
||||
import com.actiontech.dble.backend.mysql.nio.handler.builder.sqlvisitor.PushDownVisitor;
|
||||
import com.actiontech.dble.backend.mysql.nio.handler.query.DMLResponseHandler;
|
||||
import com.actiontech.dble.config.ErrorCode;
|
||||
import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.config.model.TableConfig;
|
||||
import com.actiontech.dble.config.model.TableConfig.TableTypeEnum;
|
||||
import com.actiontech.dble.plan.common.exception.MySQLOutPutException;
|
||||
import com.actiontech.dble.plan.common.item.Item;
|
||||
import com.actiontech.dble.plan.node.TableNode;
|
||||
import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.RouteResultsetNode;
|
||||
import com.actiontech.dble.server.NonBlockingSession;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.*;
|
||||
|
||||
class TableNodeHandlerBuilder extends BaseHandlerBuilder {
|
||||
private TableNode node;
|
||||
private TableConfig tableConfig = null;
|
||||
|
||||
protected TableNodeHandlerBuilder(NonBlockingSession session, TableNode node, HandlerBuilder hBuilder, boolean isExplain) {
|
||||
TableNodeHandlerBuilder(NonBlockingSession session, TableNode node, HandlerBuilder hBuilder, boolean isExplain) {
|
||||
super(session, node, hBuilder, isExplain);
|
||||
this.node = node;
|
||||
this.canPushDown = !node.existUnPushDownGroup();
|
||||
@@ -48,19 +49,22 @@ class TableNodeHandlerBuilder extends BaseHandlerBuilder {
|
||||
PushDownVisitor pdVisitor = new PushDownVisitor(node, true);
|
||||
MergeBuilder mergeBuilder = new MergeBuilder(session, node, needCommon, pdVisitor);
|
||||
String sql = null;
|
||||
Map<String, String> mapTableToSimple = new HashMap<>();
|
||||
if (node.getAst() != null && node.getParent() == null) { // it's root
|
||||
pdVisitor.visit();
|
||||
sql = pdVisitor.getSql().toString();
|
||||
mapTableToSimple = pdVisitor.getMapTableToSimple();
|
||||
}
|
||||
RouteResultsetNode[] rrssArray;
|
||||
SchemaConfig schemaConfig = DbleServer.getInstance().getConfig().getSchemas().get(node.getSchema());
|
||||
// maybe some node is view
|
||||
RouteResultset rrs;
|
||||
if (sql == null) {
|
||||
rrssArray = mergeBuilder.construct().getNodes();
|
||||
rrs = mergeBuilder.construct(schemaConfig);
|
||||
} else {
|
||||
rrssArray = mergeBuilder.constructByStatement(sql, node.getAst()).getNodes();
|
||||
rrs = mergeBuilder.constructByStatement(sql, mapTableToSimple, node.getAst(), schemaConfig);
|
||||
}
|
||||
this.needCommon = mergeBuilder.getNeedCommonFlag();
|
||||
buildMergeHandler(node, rrssArray);
|
||||
buildMergeHandler(node, rrs.getNodes());
|
||||
} catch (Exception e) {
|
||||
throw new MySQLOutPutException(ErrorCode.ER_QUERYHANDLER, "", "table node buildOwn exception! Error:" + e.getMessage(), e);
|
||||
}
|
||||
@@ -75,10 +79,11 @@ class TableNodeHandlerBuilder extends BaseHandlerBuilder {
|
||||
throw new MySQLOutPutException(ErrorCode.ER_QUERYHANDLER, "", "unexpected exception!");
|
||||
List<RouteResultsetNode> rrssList = new ArrayList<>();
|
||||
MergeBuilder mergeBuilder = new MergeBuilder(session, node, needCommon, pdVisitor);
|
||||
SchemaConfig schemaConfig = DbleServer.getInstance().getConfig().getSchemas().get(node.getSchema());
|
||||
if (tableConfig == null || tableConfig.getTableType() == TableTypeEnum.TYPE_GLOBAL_TABLE) {
|
||||
for (Item filter : filters) {
|
||||
node.setWhereFilter(filter);
|
||||
RouteResultsetNode[] rrssArray = mergeBuilder.construct().getNodes();
|
||||
RouteResultsetNode[] rrssArray = mergeBuilder.construct(schemaConfig).getNodes();
|
||||
rrssList.addAll(Arrays.asList(rrssArray));
|
||||
}
|
||||
if (filters.size() == 1) {
|
||||
@@ -89,7 +94,7 @@ class TableNodeHandlerBuilder extends BaseHandlerBuilder {
|
||||
for (Item filter : filters) {
|
||||
node.setWhereFilter(filter);
|
||||
pdVisitor.visit();
|
||||
RouteResultsetNode[] rrssArray = mergeBuilder.construct().getNodes();
|
||||
RouteResultsetNode[] rrssArray = mergeBuilder.construct(schemaConfig).getNodes();
|
||||
rrssList.addAll(Arrays.asList(rrssArray));
|
||||
}
|
||||
if (tryGlobal) {
|
||||
|
||||
@@ -120,6 +120,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
PlanNode child = query.getChild();
|
||||
MysqlVisitor childVisitor = new GlobalVisitor(child, true);
|
||||
childVisitor.visit();
|
||||
mapTableToSimple.putAll(childVisitor.getMapTableToSimple());
|
||||
sqlBuilder.append(childVisitor.getSql()).append(") ").append(query.getAlias());
|
||||
if (query.isWithSubQuery() || isTopQuery) {
|
||||
buildWhere(query);
|
||||
@@ -151,6 +152,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
if (child.getChildren().size() == 0) {
|
||||
sqlBuilder.append("(");
|
||||
}
|
||||
mapTableToSimple.putAll(childVisitor.getMapTableToSimple());
|
||||
sqlBuilder.append(childVisitor.getSql());
|
||||
if (child.getChildren().size() == 0) {
|
||||
sqlBuilder.append(")");
|
||||
@@ -160,7 +162,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
if (merge.getOrderBys() != null && merge.getOrderBys().size() > 0) {
|
||||
sqlBuilder.append(" ORDER BY ");
|
||||
for (Order order : merge.getOrderBys()) {
|
||||
sqlBuilder.append(order.getItem().getItemName() + " " + order.getSortOrder() + ",");
|
||||
sqlBuilder.append(order.getItem().getItemName()).append(" ").append(order.getSortOrder()).append(",");
|
||||
}
|
||||
sqlBuilder.setLength(sqlBuilder.length() - 1);
|
||||
}
|
||||
@@ -168,7 +170,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
if (merge.getLimitTo() != -1) {
|
||||
sqlBuilder.append(" LIMIT ");
|
||||
if (merge.getLimitFrom() != -1) {
|
||||
sqlBuilder.append(merge.getLimitFrom() + ",");
|
||||
sqlBuilder.append(merge.getLimitFrom()).append(",");
|
||||
}
|
||||
sqlBuilder.append(merge.getLimitTo());
|
||||
}
|
||||
@@ -186,6 +188,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
PlanNode left = join.getLeftNode();
|
||||
MysqlVisitor leftVisitor = new GlobalVisitor(left, false);
|
||||
leftVisitor.visit();
|
||||
mapTableToSimple.putAll(leftVisitor.getMapTableToSimple());
|
||||
sqlBuilder.append(leftVisitor.getSql());
|
||||
if (join.getLeftOuter() && join.getRightOuter()) {
|
||||
throw new RuntimeException("not supported for full outer join");
|
||||
@@ -200,6 +203,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
PlanNode right = join.getRightNode();
|
||||
MysqlVisitor rightVisitor = new GlobalVisitor(right, false);
|
||||
rightVisitor.visit();
|
||||
mapTableToSimple.putAll(rightVisitor.getMapTableToSimple());
|
||||
sqlBuilder.append(rightVisitor.getSql());
|
||||
StringBuilder joinOnFilterStr = new StringBuilder();
|
||||
boolean first = true;
|
||||
@@ -236,7 +240,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
}
|
||||
|
||||
|
||||
protected void buildSelect(PlanNode query) {
|
||||
private void buildSelect(PlanNode query) {
|
||||
sqlBuilder.append("select ");
|
||||
boolean hasDistinct = query.isDistinct();
|
||||
boolean first = true;
|
||||
@@ -254,7 +258,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
sqlBuilder.append(sb);
|
||||
}
|
||||
|
||||
protected void buildGroupBy(PlanNode query) {
|
||||
private void buildGroupBy(PlanNode query) {
|
||||
boolean first = true;
|
||||
if (query.getGroupBys() != null && query.getGroupBys().size() > 0) {
|
||||
sqlBuilder.append(" GROUP BY ");
|
||||
@@ -274,7 +278,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
}
|
||||
}
|
||||
|
||||
protected void buildHaving(PlanNode query) {
|
||||
private void buildHaving(PlanNode query) {
|
||||
if (query.getHavingFilter() != null) {
|
||||
Item filter = query.getHavingFilter();
|
||||
String pdName = visitUnSelPushDownName(filter, true);
|
||||
@@ -282,7 +286,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
}
|
||||
}
|
||||
|
||||
protected void buildOrderBy(PlanNode query) {
|
||||
private void buildOrderBy(PlanNode query) {
|
||||
boolean first = true;
|
||||
if (query.getOrderBys() != null && !query.getOrderBys().isEmpty()) {
|
||||
sqlBuilder.append(" order by ");
|
||||
@@ -304,7 +308,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
}
|
||||
}
|
||||
|
||||
protected void buildLimit(PlanNode query) {
|
||||
private void buildLimit(PlanNode query) {
|
||||
long limitFrom = query.getLimitFrom();
|
||||
long limitTo = query.getLimitTo();
|
||||
if (limitFrom == -1 && limitTo == -1) {
|
||||
@@ -394,6 +398,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
builder.append("(");
|
||||
builder.append(childVisitor.getSql());
|
||||
builder.append(")");
|
||||
mapTableToSimple.putAll(childVisitor.getMapTableToSimple());
|
||||
return builder.toString();
|
||||
} else if (item instanceof ItemExistsSubQuery) {
|
||||
ItemExistsSubQuery existsSubQuery = (ItemExistsSubQuery) item;
|
||||
@@ -408,6 +413,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
builder.append("(");
|
||||
builder.append(childVisitor.getSql());
|
||||
builder.append(")");
|
||||
mapTableToSimple.putAll(childVisitor.getMapTableToSimple());
|
||||
return builder.toString();
|
||||
} else if (item instanceof ItemCondAnd || item instanceof ItemCondOr) {
|
||||
String cond;
|
||||
@@ -434,19 +440,14 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
PlanNode child = item.getPlanNode();
|
||||
MysqlVisitor childVisitor = new GlobalVisitor(child, true);
|
||||
childVisitor.visit();
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("(");
|
||||
builder.append(childVisitor.getSql());
|
||||
builder.append(")");
|
||||
return builder.toString();
|
||||
mapTableToSimple.putAll(childVisitor.getMapTableToSimple());
|
||||
return "(" + childVisitor.getSql() + ")";
|
||||
}
|
||||
|
||||
private String buildCmpSubQueryItem(ItemBoolFunc2 item, boolean canUseAlias) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append(buildCmpArgSubQueryItem(item.arguments().get(0), canUseAlias));
|
||||
builder.append(item.funcName());
|
||||
builder.append(buildCmpArgSubQueryItem(item.arguments().get(1), canUseAlias));
|
||||
return builder.toString();
|
||||
return buildCmpArgSubQueryItem(item.arguments().get(0), canUseAlias) +
|
||||
item.funcName() +
|
||||
buildCmpArgSubQueryItem(item.arguments().get(1), canUseAlias);
|
||||
}
|
||||
|
||||
private String buildCmpArgSubQueryItem(Item arg, boolean canUseAlias) {
|
||||
@@ -467,6 +468,7 @@ public class GlobalVisitor extends MysqlVisitor {
|
||||
builder.append("(");
|
||||
builder.append(childVisitor.getSql());
|
||||
builder.append(")");
|
||||
mapTableToSimple.putAll(childVisitor.getMapTableToSimple());
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -8,6 +8,7 @@ package com.actiontech.dble.backend.mysql.nio.handler.builder.sqlvisitor;
|
||||
import com.actiontech.dble.plan.common.item.Item;
|
||||
import com.actiontech.dble.plan.common.item.Item.ItemType;
|
||||
import com.actiontech.dble.plan.common.item.function.operator.ItemBoolFunc2;
|
||||
import com.actiontech.dble.plan.common.item.function.operator.cmpfunc.ItemFuncIn;
|
||||
import com.actiontech.dble.plan.common.item.function.operator.logic.ItemCondAnd;
|
||||
import com.actiontech.dble.plan.common.item.function.operator.logic.ItemCondOr;
|
||||
import com.actiontech.dble.plan.common.item.function.operator.logic.ItemFuncNot;
|
||||
@@ -31,22 +32,26 @@ import java.util.Map;
|
||||
*/
|
||||
public abstract class MysqlVisitor {
|
||||
// the max column size of mysql
|
||||
protected static final int MAX_COL_LENGTH = 255;
|
||||
static final int MAX_COL_LENGTH = 255;
|
||||
// map :sel name->push name
|
||||
protected Map<String, String> pushNameMap = new HashMap<>();
|
||||
protected boolean isTopQuery = false;
|
||||
Map<String, String> pushNameMap = new HashMap<>();
|
||||
boolean isTopQuery = false;
|
||||
protected PlanNode query;
|
||||
protected long randomIndex = 0L;
|
||||
private long randomIndex = 0L;
|
||||
|
||||
/* is all function can be push down?if not,it need to calc by middle-ware */
|
||||
protected boolean existUnPushDownGroup = false;
|
||||
protected boolean visited = false;
|
||||
boolean existUnPushDownGroup = false;
|
||||
boolean visited = false;
|
||||
// -- start replaceable string builder
|
||||
protected ReplaceableStringBuilder replaceableSqlBuilder = new ReplaceableStringBuilder();
|
||||
ReplaceableStringBuilder replaceableSqlBuilder = new ReplaceableStringBuilder();
|
||||
// tmp sql
|
||||
protected StringBuilder sqlBuilder;
|
||||
protected StringPtr replaceableWhere = new StringPtr("");
|
||||
protected Item whereFilter = null;
|
||||
public MysqlVisitor(PlanNode query, boolean isTopQuery) {
|
||||
StringBuilder sqlBuilder;
|
||||
StringPtr replaceableWhere = new StringPtr("");
|
||||
Item whereFilter = null;
|
||||
|
||||
Map<String, String> mapTableToSimple = new HashMap<>();
|
||||
|
||||
MysqlVisitor(PlanNode query, boolean isTopQuery) {
|
||||
this.query = query;
|
||||
this.isTopQuery = isTopQuery;
|
||||
}
|
||||
@@ -55,11 +60,18 @@ public abstract class MysqlVisitor {
|
||||
return replaceableSqlBuilder;
|
||||
}
|
||||
|
||||
public Map<String, String> getMapTableToSimple() {
|
||||
return mapTableToSimple;
|
||||
}
|
||||
|
||||
public abstract void visit();
|
||||
|
||||
|
||||
protected void buildTableName(TableNode tableNode, StringBuilder sb) {
|
||||
sb.append(" `").append(tableNode.getPureName()).append("`");
|
||||
void buildTableName(TableNode tableNode, StringBuilder sb) {
|
||||
String tableName = "`" + tableNode.getPureName() + "`";
|
||||
String fullName = "`" + tableNode.getPureSchema() + "`." + tableName;
|
||||
mapTableToSimple.put(fullName, tableName);
|
||||
sb.append(" ").append(fullName);
|
||||
String alias = tableNode.getAlias();
|
||||
if (alias != null) {
|
||||
sb.append(" `").append(alias).append("`");
|
||||
@@ -81,7 +93,7 @@ public abstract class MysqlVisitor {
|
||||
}
|
||||
|
||||
/* change where to replaceable */
|
||||
protected void buildWhere(PlanNode planNode) {
|
||||
void buildWhere(PlanNode planNode) {
|
||||
if (!visited)
|
||||
replaceableSqlBuilder.getCurrentElement().setRepString(replaceableWhere);
|
||||
StringBuilder whereBuilder = new StringBuilder();
|
||||
@@ -101,7 +113,7 @@ public abstract class MysqlVisitor {
|
||||
return "_$" + aggFuncName + "$_";
|
||||
}
|
||||
|
||||
protected String getRandomAliasName() {
|
||||
String getRandomAliasName() {
|
||||
return "rpda_" + randomIndex++;
|
||||
}
|
||||
|
||||
@@ -159,7 +171,37 @@ public abstract class MysqlVisitor {
|
||||
Item b = item.arguments().get(1);
|
||||
return getItemName(a) + " " + ((ItemBoolFunc2) item).funcName() + " " + getItemName(b);
|
||||
} else if (item.type().equals(ItemType.FIELD_ITEM)) {
|
||||
return "`" + item.getTableName() + "`.`" + item.getItemName() + "`";
|
||||
String tableName = "`" + item.getTableName() + "`.`" + item.getItemName() + "`";
|
||||
if (item.getDbName() == null) {
|
||||
return tableName;
|
||||
}
|
||||
if (item.getReferTables().size() == 0) {
|
||||
return tableName;
|
||||
}
|
||||
PlanNode tbNode = item.getReferTables().iterator().next();
|
||||
if (!(tbNode instanceof TableNode)) {
|
||||
return tableName;
|
||||
}
|
||||
if (!((TableNode) tbNode).getTableName().equals(item.getTableName())) {
|
||||
return tableName;
|
||||
}
|
||||
return "`" + item.getDbName() + "`." + tableName;
|
||||
} else if (item instanceof ItemFuncIn) {
|
||||
Item a = item.arguments().get(0);
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append(getItemName(a));
|
||||
if (((ItemFuncIn) item).isNegate()) {
|
||||
sb.append(" not ");
|
||||
}
|
||||
sb.append(" in (");
|
||||
for (int index = 1; index < item.arguments().size(); index++) {
|
||||
if (index > 1) {
|
||||
sb.append(",");
|
||||
}
|
||||
sb.append(getItemName(item.arguments().get(index)));
|
||||
}
|
||||
sb.append(")");
|
||||
return sb.toString();
|
||||
} else {
|
||||
return item.getItemName();
|
||||
}
|
||||
|
||||
@@ -112,6 +112,7 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
PlanNode left = join.getLeftNode();
|
||||
MysqlVisitor leftVisitor = new GlobalVisitor(left, false);
|
||||
leftVisitor.visit();
|
||||
mapTableToSimple.putAll(leftVisitor.getMapTableToSimple());
|
||||
replaceableSqlBuilder.append(leftVisitor.getSql());
|
||||
sqlBuilder = replaceableSqlBuilder.getCurrentElement().getSb();
|
||||
if (join.getLeftOuter() && join.getRightOuter()) {
|
||||
@@ -127,6 +128,7 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
PlanNode right = join.getRightNode();
|
||||
MysqlVisitor rightVisitor = new GlobalVisitor(right, false);
|
||||
rightVisitor.visit();
|
||||
mapTableToSimple.putAll(rightVisitor.getMapTableToSimple());
|
||||
replaceableSqlBuilder.append(rightVisitor.getSql());
|
||||
sqlBuilder = replaceableSqlBuilder.getCurrentElement().getSb();
|
||||
StringBuilder joinOnFilterStr = getJoinOn(join, leftVisitor, rightVisitor);
|
||||
@@ -191,7 +193,7 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
return joinOnFilterStr;
|
||||
}
|
||||
|
||||
protected void buildWhere(JoinNode planNode, Item leftFilter, Item rightFilter) {
|
||||
private void buildWhere(JoinNode planNode, Item leftFilter, Item rightFilter) {
|
||||
if (!visited)
|
||||
replaceableSqlBuilder.getCurrentElement().setRepString(replaceableWhere);
|
||||
StringBuilder whereBuilder = new StringBuilder();
|
||||
@@ -221,7 +223,7 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
sqlBuilder = replaceableSqlBuilder.getCurrentElement().getSb();
|
||||
}
|
||||
|
||||
protected void buildSelect(PlanNode query) {
|
||||
private void buildSelect(PlanNode query) {
|
||||
sqlBuilder.append("select ");
|
||||
if (query.isDistinct()) {
|
||||
sqlBuilder.append("DISTINCT ");
|
||||
@@ -236,13 +238,12 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
continue;
|
||||
if ((col.type().equals(Item.ItemType.FUNC_ITEM) || col.type().equals(Item.ItemType.COND_ITEM)) && col.isWithSumFunc())
|
||||
continue;
|
||||
String pdName = visitPushDownNameSel(col);
|
||||
if (StringUtils.isEmpty(pdName))// it's null when duplicate column
|
||||
final String colName = visitPushDownNameSel(col);
|
||||
if (StringUtils.isEmpty(colName))// it's null when duplicate column
|
||||
continue;
|
||||
if (col.type().equals(Item.ItemType.SUM_FUNC_ITEM)) {
|
||||
ItemSum funCol = (ItemSum) col;
|
||||
String funName = funCol.funcName().toUpperCase();
|
||||
String colName = pdName;
|
||||
ItemSum.SumFuncType i = funCol.sumType();
|
||||
if (i == ItemSum.SumFuncType.AVG_FUNC) {
|
||||
String colNameSum = colName.replace(funName + "(", "SUM(");
|
||||
@@ -252,7 +253,7 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
sqlBuilder.append(colNameSum).append(",").append(colNameCount).append(",");
|
||||
continue;
|
||||
} else if (i == ItemSum.SumFuncType.STD_FUNC || i == ItemSum.SumFuncType.VARIANCE_FUNC) {
|
||||
String toReplace = "";
|
||||
String toReplace;
|
||||
if (i == ItemSum.SumFuncType.STD_FUNC) {
|
||||
toReplace = "(STDDEV_SAMP\\()|(STDDEV_POP\\()|(STDDEV\\()|(STD\\()";
|
||||
} else {
|
||||
@@ -268,13 +269,13 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
sqlBuilder.append(pdName);
|
||||
sqlBuilder.append(colName);
|
||||
sqlBuilder.append(",");
|
||||
}
|
||||
sqlBuilder.deleteCharAt(sqlBuilder.length() - 1);
|
||||
}
|
||||
|
||||
protected void buildGroupBy(PlanNode query) {
|
||||
private void buildGroupBy(PlanNode query) {
|
||||
if (nodeHasGroupBy(query)) {
|
||||
// push down group by
|
||||
if (!existUnPushDownGroup) {
|
||||
@@ -316,7 +317,7 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
}
|
||||
}
|
||||
|
||||
protected void buildOrderBy(PlanNode query) {
|
||||
private void buildOrderBy(PlanNode query) {
|
||||
/* if group by exists,it must merge as "group by"'s order,so don't push down order */
|
||||
boolean realPush = query.getGroupBys().isEmpty();
|
||||
if (query.getOrderBys().size() > 0) {
|
||||
@@ -343,7 +344,7 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
}
|
||||
}
|
||||
|
||||
protected void buildLimit(PlanNode query, StringBuilder sb) {
|
||||
private void buildLimit(PlanNode query, StringBuilder sb) {
|
||||
/* both group by and limit are exist, don't push down limit */
|
||||
if (query.getGroupBys().isEmpty() && !existUnPushDownGroup) {
|
||||
if (query.getLimitFrom() != -1 && query.getLimitTo() != -1) {
|
||||
@@ -355,13 +356,13 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
|
||||
/* -------------------------- help method ------------------------ */
|
||||
|
||||
public static boolean nodeHasGroupBy(PlanNode node) {
|
||||
private static boolean nodeHasGroupBy(PlanNode node) {
|
||||
return (node.getSumFuncs().size() > 0 || node.getGroupBys().size() > 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String visitPushDownNameSel(Item item) {
|
||||
String orgPushDownName = null;
|
||||
String orgPushDownName;
|
||||
if (item.isWithSubQuery()) {
|
||||
Item tmpItem = PlanUtil.rebuildSubQueryItem(item);
|
||||
orgPushDownName = tmpItem.getItemName();
|
||||
@@ -408,7 +409,7 @@ public class PushDownVisitor extends MysqlVisitor {
|
||||
return orgPushDownName + " as `" + pushAlias + "`";
|
||||
}
|
||||
|
||||
protected void buildForUpdate(TableNode query, StringBuilder sb) {
|
||||
private void buildForUpdate(TableNode query, StringBuilder sb) {
|
||||
if (query.getAst() != null) {
|
||||
SQLSelectQuery queryblock = query.getAst().getSelect().getQuery();
|
||||
if (queryblock instanceof MySqlSelectQueryBlock) {
|
||||
|
||||
@@ -427,7 +427,7 @@ public class ConfigInitializer implements ProblemReporter {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
boolean isConnected = ds.testConnection(schema);
|
||||
boolean isConnected = ds.testConnection();
|
||||
boolPtr.set(isConnected);
|
||||
} catch (IOException e) {
|
||||
boolPtr.set(false);
|
||||
|
||||
@@ -44,7 +44,10 @@ public final class SelectHandler {
|
||||
SelectVersionComment.response(c);
|
||||
break;
|
||||
case SESSION_TX_READ_ONLY:
|
||||
SelectSessionTxReadOnly.execute(c);
|
||||
SelectSessionTxReadOnly.execute(c, stmt.substring(offset).trim());
|
||||
break;
|
||||
case SESSION_TRANSACTION_READ_ONLY:
|
||||
SelectSessionTxReadOnly.execute(c, stmt.substring(offset).trim());
|
||||
break;
|
||||
case MAX_ALLOWED_PACKET:
|
||||
SelectMaxAllowedPacket.execute(c);
|
||||
|
||||
@@ -8,12 +8,9 @@ package com.actiontech.dble.manager.response;
|
||||
import com.actiontech.dble.config.ErrorCode;
|
||||
import com.actiontech.dble.manager.ManagerConnection;
|
||||
import com.actiontech.dble.net.mysql.OkPacket;
|
||||
import com.actiontech.dble.statistic.stat.UserStat;
|
||||
import com.actiontech.dble.statistic.stat.UserStatAnalyzer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public final class ReloadSqlSlowTime {
|
||||
private ReloadSqlSlowTime() {
|
||||
@@ -27,10 +24,10 @@ public final class ReloadSqlSlowTime {
|
||||
return;
|
||||
}
|
||||
|
||||
Map<String, UserStat> statMap = UserStatAnalyzer.getInstance().getUserStatMap();
|
||||
for (UserStat userStat : statMap.values()) {
|
||||
userStat.setSlowTime(time);
|
||||
}
|
||||
//Map<String, UserStat> statMap = UserStatAnalyzer.getInstance().getUserStatMap();
|
||||
//for (UserStat userStat : statMap.values()) {
|
||||
// userStat.setSlowTime(time);
|
||||
//}
|
||||
|
||||
LOGGER.info(String.valueOf(c) + " reload @@sqlslow=" + time + " success by manager");
|
||||
|
||||
|
||||
@@ -20,7 +20,7 @@ public final class SelectMaxAllowedPacket {
|
||||
private SelectMaxAllowedPacket() {
|
||||
}
|
||||
|
||||
private static final String SESSION_TX_READ_ONLY = "@@max_allowed_packet";
|
||||
private static final String MAX_ALLOWED_PACKET = "@@max_allowed_packet";
|
||||
private static final int FIELD_COUNT = 1;
|
||||
private static final ResultSetHeaderPacket HEADER = PacketUtil.getHeader(FIELD_COUNT);
|
||||
private static final FieldPacket[] FIELDS = new FieldPacket[FIELD_COUNT];
|
||||
@@ -31,7 +31,7 @@ public final class SelectMaxAllowedPacket {
|
||||
byte packetId = 0;
|
||||
HEADER.setPacketId(++packetId);
|
||||
|
||||
FIELDS[i] = PacketUtil.getField(SESSION_TX_READ_ONLY, Fields.FIELD_TYPE_INT24);
|
||||
FIELDS[i] = PacketUtil.getField(MAX_ALLOWED_PACKET, Fields.FIELD_TYPE_INT24);
|
||||
FIELDS[i].setPacketId(++packetId);
|
||||
|
||||
EOF.setPacketId(++packetId);
|
||||
|
||||
@@ -20,40 +20,29 @@ public final class SelectSessionTxReadOnly {
|
||||
private SelectSessionTxReadOnly() {
|
||||
}
|
||||
|
||||
private static final String SESSION_TX_READ_ONLY = "@@SESSION.TX_READ_ONLY";
|
||||
private static final int FIELD_COUNT = 1;
|
||||
private static final ResultSetHeaderPacket HEADER = PacketUtil.getHeader(FIELD_COUNT);
|
||||
private static final FieldPacket[] FIELDS = new FieldPacket[FIELD_COUNT];
|
||||
private static final EOFPacket EOF = new EOFPacket();
|
||||
|
||||
static {
|
||||
int i = 0;
|
||||
byte packetId = 0;
|
||||
HEADER.setPacketId(++packetId);
|
||||
|
||||
FIELDS[i] = PacketUtil.getField(SESSION_TX_READ_ONLY, Fields.FIELD_TYPE_INT24);
|
||||
FIELDS[i].setPacketId(++packetId);
|
||||
|
||||
EOF.setPacketId(++packetId);
|
||||
}
|
||||
|
||||
public static void execute(ManagerConnection c) {
|
||||
public static void execute(ManagerConnection c, String column) {
|
||||
ByteBuffer buffer = c.allocate();
|
||||
|
||||
byte packetId = 0;
|
||||
ResultSetHeaderPacket header = PacketUtil.getHeader(1);
|
||||
header.setPacketId(++packetId);
|
||||
// write header
|
||||
buffer = HEADER.write(buffer, c, true);
|
||||
buffer = header.write(buffer, c, true);
|
||||
FieldPacket[] fields = new FieldPacket[1];
|
||||
fields[0] = PacketUtil.getField(column, Fields.FIELD_TYPE_INT24);
|
||||
fields[0].setPacketId(++packetId);
|
||||
|
||||
// write fields
|
||||
for (FieldPacket field : FIELDS) {
|
||||
for (FieldPacket field : fields) {
|
||||
buffer = field.write(buffer, c, true);
|
||||
}
|
||||
|
||||
EOFPacket eof = new EOFPacket();
|
||||
eof.setPacketId(++packetId);
|
||||
// write eof
|
||||
buffer = EOF.write(buffer, c, true);
|
||||
buffer = eof.write(buffer, c, true);
|
||||
|
||||
// write rows
|
||||
byte packetId = EOF.getPacketId();
|
||||
RowDataPacket row = new RowDataPacket(FIELD_COUNT);
|
||||
RowDataPacket row = new RowDataPacket(1);
|
||||
row.setPacketId(++packetId);
|
||||
row.add(LongUtil.toBytes(0));
|
||||
buffer = row.write(buffer, c, true);
|
||||
|
||||
@@ -16,6 +16,7 @@ import com.actiontech.dble.statistic.stat.SqlFrequency;
|
||||
import com.actiontech.dble.statistic.stat.UserStat;
|
||||
import com.actiontech.dble.statistic.stat.UserStatAnalyzer;
|
||||
import com.actiontech.dble.util.FormatUtil;
|
||||
import com.actiontech.dble.util.IntegerUtil;
|
||||
import com.actiontech.dble.util.LongUtil;
|
||||
import com.actiontech.dble.util.StringUtil;
|
||||
|
||||
@@ -32,7 +33,7 @@ public final class ShowSQLHigh {
|
||||
private ShowSQLHigh() {
|
||||
}
|
||||
|
||||
private static final int FIELD_COUNT = 9;
|
||||
private static final int FIELD_COUNT = 11;
|
||||
private static final ResultSetHeaderPacket HEADER = PacketUtil.getHeader(FIELD_COUNT);
|
||||
private static final FieldPacket[] FIELDS = new FieldPacket[FIELD_COUNT];
|
||||
private static final EOFPacket EOF = new EOFPacket();
|
||||
@@ -51,6 +52,8 @@ public final class ShowSQLHigh {
|
||||
FIELDS[i] = PacketUtil.getField("FREQUENCY", Fields.FIELD_TYPE_LONGLONG);
|
||||
FIELDS[i++].setPacketId(++packetId);
|
||||
|
||||
FIELDS[i] = PacketUtil.getField("TOTAL_TIME", Fields.FIELD_TYPE_LONGLONG);
|
||||
FIELDS[i++].setPacketId(++packetId);
|
||||
FIELDS[i] = PacketUtil.getField("AVG_TIME", Fields.FIELD_TYPE_LONGLONG);
|
||||
FIELDS[i++].setPacketId(++packetId);
|
||||
FIELDS[i] = PacketUtil.getField("MAX_TIME", Fields.FIELD_TYPE_LONGLONG);
|
||||
@@ -66,6 +69,9 @@ public final class ShowSQLHigh {
|
||||
FIELDS[i] = PacketUtil.getField("SQL", Fields.FIELD_TYPE_VAR_STRING);
|
||||
FIELDS[i++].setPacketId(++packetId);
|
||||
|
||||
FIELDS[i] = PacketUtil.getField("digest", Fields.FIELD_TYPE_INT24);
|
||||
FIELDS[i++].setPacketId(++packetId);
|
||||
|
||||
EOF.setPacketId(++packetId);
|
||||
}
|
||||
|
||||
@@ -95,8 +101,8 @@ public final class ShowSQLHigh {
|
||||
for (SqlFrequency sqlFrequency : list) {
|
||||
if (sqlFrequency != null) {
|
||||
RowDataPacket row = getRow(i, user, sqlFrequency.getSql(), sqlFrequency.getCount(),
|
||||
sqlFrequency.getAvgTime(), sqlFrequency.getMaxTime(), sqlFrequency.getMinTime(),
|
||||
sqlFrequency.getExecuteTime(), sqlFrequency.getLastTime(), c.getCharset().getResults());
|
||||
sqlFrequency.getAllExecuteTime(), sqlFrequency.getAvgTime(), sqlFrequency.getMaxTime(), sqlFrequency.getMinTime(),
|
||||
sqlFrequency.getExecuteTime(), sqlFrequency.getLastTime(), c.getCharset().getResults(), sqlFrequency.getSql().hashCode());
|
||||
row.setPacketId(++packetId);
|
||||
buffer = row.write(buffer, c, true);
|
||||
i++;
|
||||
@@ -114,18 +120,20 @@ public final class ShowSQLHigh {
|
||||
c.write(buffer);
|
||||
}
|
||||
|
||||
private static RowDataPacket getRow(int i, String user, String sql, long count, long avgTime, long maxTime,
|
||||
long minTime, long executeTime, long lastTime, String charset) {
|
||||
private static RowDataPacket getRow(int i, String user, String sql, long count, long allExecuteTime, long avgTime, long maxTime,
|
||||
long minTime, long executeTime, long lastTime, String charset, int hashcode) {
|
||||
RowDataPacket row = new RowDataPacket(FIELD_COUNT);
|
||||
row.add(LongUtil.toBytes(i));
|
||||
row.add(StringUtil.encode(user, charset));
|
||||
row.add(LongUtil.toBytes(count));
|
||||
row.add(LongUtil.toBytes(allExecuteTime));
|
||||
row.add(LongUtil.toBytes(avgTime));
|
||||
row.add(LongUtil.toBytes(maxTime));
|
||||
row.add(LongUtil.toBytes(minTime));
|
||||
row.add(LongUtil.toBytes(executeTime));
|
||||
row.add(StringUtil.encode(FormatUtil.formatDate(lastTime), charset));
|
||||
row.add(StringUtil.encode(sql, charset));
|
||||
row.add(IntegerUtil.toBytes(hashcode));
|
||||
return row;
|
||||
}
|
||||
|
||||
|
||||
16
src/main/java/com/actiontech/dble/net/mysql/HandshakeV10Packet.java
Executable file → Normal file
16
src/main/java/com/actiontech/dble/net/mysql/HandshakeV10Packet.java
Executable file → Normal file
@@ -58,25 +58,25 @@ public class HandshakeV10Packet extends MySQLPacket {
|
||||
public static final byte[] NATIVE_PASSWORD_PLUGIN = "mysql_native_password".getBytes();
|
||||
public static final byte[] CACHING_SHA2_PASSWORD_PLUGIN = "caching_sha2_password".getBytes();
|
||||
|
||||
|
||||
private byte protocolVersion;
|
||||
private byte[] serverVersion;
|
||||
private long threadId;
|
||||
private byte[] seed; // auth-plugin-data-part-1
|
||||
|
||||
|
||||
|
||||
public byte[] getAuthPluginName() {
|
||||
return authPluginName;
|
||||
}
|
||||
|
||||
|
||||
private int serverCapabilities;
|
||||
private byte serverCharsetIndex;
|
||||
private int serverStatus;
|
||||
private byte[] restOfScrambleBuff; // auth-plugin-data-part-2
|
||||
private byte[] authPluginName = NATIVE_PASSWORD_PLUGIN;
|
||||
|
||||
public byte[] getAuthPluginName() {
|
||||
return authPluginName;
|
||||
}
|
||||
|
||||
public byte[] getServerVersion() {
|
||||
return serverVersion;
|
||||
}
|
||||
|
||||
public void write(FrontendConnection c) {
|
||||
ByteBuffer buffer = c.allocate();
|
||||
BufferUtil.writeUB3(buffer, calcPacketSize());
|
||||
|
||||
@@ -34,4 +34,8 @@ public abstract class ItemFuncOptNeg extends ItemIntFunc {
|
||||
negated = !negated;
|
||||
}
|
||||
|
||||
public boolean isNegate() {
|
||||
return negated;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -33,7 +33,7 @@ public final class RouteResultset implements Serializable {
|
||||
|
||||
private boolean needOptimizer;
|
||||
private int limitStart;
|
||||
private boolean cacheAble;
|
||||
private boolean sqlRouteCacheAble;
|
||||
// used to store table's ID->data nodes cache
|
||||
private String primaryKey;
|
||||
private boolean containsPrimaryFilter = false;
|
||||
@@ -139,12 +139,12 @@ public final class RouteResultset implements Serializable {
|
||||
}
|
||||
|
||||
|
||||
public boolean isCacheAble() {
|
||||
return cacheAble;
|
||||
public boolean isSqlRouteCacheAble() {
|
||||
return sqlRouteCacheAble;
|
||||
}
|
||||
|
||||
public void setCacheAble(boolean cacheAble) {
|
||||
this.cacheAble = cacheAble;
|
||||
public void setSqlRouteCacheAble(boolean sqlRouteCacheAble) {
|
||||
this.sqlRouteCacheAble = sqlRouteCacheAble;
|
||||
}
|
||||
|
||||
public int getSqlType() {
|
||||
|
||||
@@ -112,7 +112,7 @@ public class RouteService {
|
||||
rrs = RouteStrategyFactory.getRouteStrategy().route(schema, sqlType, stmt, sc, tableId2DataNodeCache);
|
||||
}
|
||||
|
||||
if (rrs != null && sqlType == ServerParse.SELECT && rrs.isCacheAble() && !LOGGER.isDebugEnabled() && sqlRouteCache != null &&
|
||||
if (rrs != null && sqlType == ServerParse.SELECT && rrs.isSqlRouteCacheAble() && !LOGGER.isDebugEnabled() && sqlRouteCache != null &&
|
||||
sc.getSession2().getRemingSql() == null) {
|
||||
sqlRouteCache.putIfAbsent(cacheKey, rrs);
|
||||
}
|
||||
|
||||
@@ -60,7 +60,7 @@ public abstract class AbstractRouteStrategy implements RouteStrategy {
|
||||
* debug mode and load data ,no cache
|
||||
*/
|
||||
if (LOGGER.isDebugEnabled() && origSQL.startsWith(LoadData.LOAD_DATA_HINT)) {
|
||||
rrs.setCacheAble(false);
|
||||
rrs.setSqlRouteCacheAble(false);
|
||||
}
|
||||
|
||||
if (sqlType == ServerParse.CALL) {
|
||||
|
||||
@@ -5,6 +5,7 @@
|
||||
|
||||
package com.actiontech.dble.route.parser;
|
||||
|
||||
import com.actiontech.dble.backend.mysql.VersionUtil;
|
||||
import com.actiontech.dble.route.parser.util.ParseUtil;
|
||||
|
||||
|
||||
@@ -17,9 +18,12 @@ public final class ManagerParseSelect {
|
||||
public static final int SESSION_TX_READ_ONLY = 2;
|
||||
public static final int MAX_ALLOWED_PACKET = 3;
|
||||
public static final int TIMEDIFF = 4;
|
||||
public static final int SESSION_TRANSACTION_READ_ONLY = 5;
|
||||
|
||||
private static final char[] STRING_VERSION_COMMENT = "VERSION_COMMENT".toCharArray();
|
||||
private static final char[] STRING_SESSION_TX_READ_ONLY = "SESSION.TX_READ_ONLY".toCharArray();
|
||||
private static final char[] STRING_SESSION = "SESSION.".toCharArray();
|
||||
private static final char[] STRING_SESSION_TRANSACTION_READ_ONLY = VersionUtil.TRANSACTION_READ_ONLY.toUpperCase().toCharArray();
|
||||
private static final char[] STRING_SESSION_TX_READ_ONLY = VersionUtil.TX_READ_ONLY.toUpperCase().toCharArray();
|
||||
private static final char[] STRING_MAX_ALLOWED_PACKET = "MAX_ALLOWED_PACKET".toCharArray();
|
||||
private static final char[] STRING_TIMEDIFF = "TIMEDIFF(NOW(), UTC_TIMESTAMP())".toCharArray();
|
||||
|
||||
@@ -98,10 +102,42 @@ public final class ManagerParseSelect {
|
||||
return OTHER;
|
||||
}
|
||||
|
||||
// SESSION.TX_READ_ONLY
|
||||
// SESSION.
|
||||
private static int select2SCheck(String stmt, int offset) {
|
||||
int length = offset + STRING_SESSION.length;
|
||||
if (stmt.length() < length + 2) { //session.tx
|
||||
return OTHER;
|
||||
}
|
||||
if (ParseUtil.compare(stmt, offset, STRING_SESSION)) {
|
||||
if (ParseUtil.isEOF(stmt, length)) {
|
||||
return OTHER;
|
||||
}
|
||||
} else {
|
||||
return OTHER;
|
||||
}
|
||||
offset = offset + STRING_SESSION.length;
|
||||
switch (stmt.charAt(offset)) {
|
||||
case 'T':
|
||||
case 't':
|
||||
switch (stmt.charAt(offset + 1)) {
|
||||
case 'X':
|
||||
case 'x':
|
||||
return select2TXCheck(stmt, offset);
|
||||
case 'R':
|
||||
case 'r':
|
||||
return select2TRCheck(stmt, offset);
|
||||
default:
|
||||
return OTHER;
|
||||
}
|
||||
default:
|
||||
return OTHER;
|
||||
}
|
||||
}
|
||||
|
||||
// SESSION.TX_READ_ONLY
|
||||
private static int select2TXCheck(String stmt, int offset) {
|
||||
int length = offset + STRING_SESSION_TX_READ_ONLY.length;
|
||||
if (stmt.length() >= (offset + STRING_SESSION_TX_READ_ONLY.length) && ParseUtil.compare(stmt, offset, STRING_SESSION_TX_READ_ONLY)) {
|
||||
if (stmt.length() >= length && ParseUtil.compare(stmt, offset, STRING_SESSION_TX_READ_ONLY)) {
|
||||
if (!ParseUtil.isEOF(stmt, length)) {
|
||||
return OTHER;
|
||||
}
|
||||
@@ -111,4 +147,17 @@ public final class ManagerParseSelect {
|
||||
return OTHER;
|
||||
}
|
||||
|
||||
// SESSION.TRANSACTION_READ_ONLY
|
||||
private static int select2TRCheck(String stmt, int offset) {
|
||||
int length = offset + STRING_SESSION_TRANSACTION_READ_ONLY.length;
|
||||
if (stmt.length() >= length && ParseUtil.compare(stmt, offset, STRING_SESSION_TRANSACTION_READ_ONLY)) {
|
||||
if (!ParseUtil.isEOF(stmt, length)) {
|
||||
return OTHER;
|
||||
}
|
||||
return SESSION_TRANSACTION_READ_ONLY;
|
||||
}
|
||||
|
||||
return OTHER;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -5,6 +5,8 @@
|
||||
|
||||
package com.actiontech.dble.route.parser.druid;
|
||||
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
@@ -19,10 +21,10 @@ public class DruidShardingParseInfo {
|
||||
|
||||
private List<RouteCalculateUnit> routeCalculateUnits = new ArrayList<>();
|
||||
|
||||
private List<String> tables = new ArrayList<>();
|
||||
private List<Pair<String, String>> tables = new ArrayList<>();
|
||||
|
||||
/**
|
||||
* key table alias, value talbe realname;
|
||||
* key table alias, value table real name;
|
||||
*/
|
||||
private Map<String, String> tableAliasMap = new LinkedHashMap<>();
|
||||
|
||||
@@ -34,17 +36,14 @@ public class DruidShardingParseInfo {
|
||||
this.tableAliasMap = tableAliasMap;
|
||||
}
|
||||
|
||||
public List<String> getTables() {
|
||||
public List<Pair<String, String>> getTables() {
|
||||
return tables;
|
||||
}
|
||||
|
||||
public void addTable(String tableName) {
|
||||
this.tables.add(tableName);
|
||||
public void addTable(Pair<String, String> table) {
|
||||
this.tables.add(table);
|
||||
}
|
||||
|
||||
public RouteCalculateUnit getRouteCalculateUnit() {
|
||||
return routeCalculateUnits.get(0);
|
||||
}
|
||||
|
||||
public List<RouteCalculateUnit> getRouteCalculateUnits() {
|
||||
return routeCalculateUnits;
|
||||
@@ -58,12 +57,11 @@ public class DruidShardingParseInfo {
|
||||
this.routeCalculateUnits.add(routeCalculateUnit);
|
||||
}
|
||||
|
||||
|
||||
public void clear() {
|
||||
public void clearRouteCalculateUnit() {
|
||||
for (RouteCalculateUnit unit : routeCalculateUnits) {
|
||||
unit.clear();
|
||||
}
|
||||
routeCalculateUnits.clear();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
@@ -5,6 +5,7 @@
|
||||
|
||||
package com.actiontech.dble.route.parser.druid;
|
||||
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.sqlengine.mpp.ColumnRoutePair;
|
||||
import com.actiontech.dble.sqlengine.mpp.IsValue;
|
||||
import com.actiontech.dble.sqlengine.mpp.RangeValue;
|
||||
@@ -23,14 +24,14 @@ import java.util.Set;
|
||||
* @copyright wonhigh.cn
|
||||
*/
|
||||
public class RouteCalculateUnit {
|
||||
private Map<String, Map<String, Set<ColumnRoutePair>>> tablesAndConditions = new LinkedHashMap<>();
|
||||
private Map<Pair<String, String>, Map<String, Set<ColumnRoutePair>>> tablesAndConditions = new LinkedHashMap<>();
|
||||
|
||||
public Map<String, Map<String, Set<ColumnRoutePair>>> getTablesAndConditions() {
|
||||
public Map<Pair<String, String>, Map<String, Set<ColumnRoutePair>>> getTablesAndConditions() {
|
||||
return tablesAndConditions;
|
||||
}
|
||||
|
||||
public void addShardingExpr(String tableName, String columnName, Object value) {
|
||||
Map<String, Set<ColumnRoutePair>> tableColumnsMap = tablesAndConditions.get(tableName);
|
||||
public void addShardingExpr(Pair<String, String> table, String columnName, Object value) {
|
||||
Map<String, Set<ColumnRoutePair>> tableColumnsMap = tablesAndConditions.get(table);
|
||||
|
||||
if (value == null) {
|
||||
// where a=null
|
||||
@@ -39,7 +40,7 @@ public class RouteCalculateUnit {
|
||||
|
||||
if (tableColumnsMap == null) {
|
||||
tableColumnsMap = new LinkedHashMap<>();
|
||||
tablesAndConditions.put(tableName, tableColumnsMap);
|
||||
tablesAndConditions.put(table, tableColumnsMap);
|
||||
}
|
||||
|
||||
String upperColName = columnName.toUpperCase();
|
||||
@@ -47,7 +48,7 @@ public class RouteCalculateUnit {
|
||||
|
||||
if (columnValues == null) {
|
||||
columnValues = new LinkedHashSet<>();
|
||||
tablesAndConditions.get(tableName).put(upperColName, columnValues);
|
||||
tablesAndConditions.get(table).put(upperColName, columnValues);
|
||||
}
|
||||
|
||||
if (value instanceof Object[]) {
|
||||
@@ -70,5 +71,29 @@ public class RouteCalculateUnit {
|
||||
tablesAndConditions.clear();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (Map.Entry<Pair<String, String>, Map<String, Set<ColumnRoutePair>>> entry : tablesAndConditions.entrySet()) {
|
||||
Pair<String, String> table = entry.getKey();
|
||||
String schemaName = table.getKey();
|
||||
String tableName = table.getValue();
|
||||
Map<String, Set<ColumnRoutePair>> columnsMap = entry.getValue();
|
||||
for (Map.Entry<String, Set<ColumnRoutePair>> columns : columnsMap.entrySet()) {
|
||||
String columnName = columns.getKey();
|
||||
Set<ColumnRoutePair> values = columns.getValue();
|
||||
for (ColumnRoutePair pair : values) {
|
||||
if (pair.colValue != null) {
|
||||
sb.append("{").append("schema:").append(schemaName).append(",table:").append(tableName);
|
||||
sb.append(",column:").append(columnName).append(",value:").append(pair.colValue).append("},");
|
||||
} else if (pair.rangeValue != null) {
|
||||
sb.append("{").append("schema:").append(schemaName).append(",table:").append(tableName);
|
||||
sb.append(",column:").append(columnName).append(",value between:").append(pair.rangeValue.getBeginValue());
|
||||
sb.append("~").append(pair.rangeValue.getEndValue()).append("},");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -6,6 +6,7 @@
|
||||
package com.actiontech.dble.route.parser.druid;
|
||||
|
||||
import com.actiontech.dble.DbleServer;
|
||||
import com.actiontech.dble.route.util.ConditionUtil;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.alibaba.druid.sql.ast.SQLExpr;
|
||||
import com.alibaba.druid.sql.ast.SQLName;
|
||||
@@ -17,6 +18,7 @@ import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlInsertStatement;
|
||||
import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlSelectQueryBlock;
|
||||
import com.alibaba.druid.sql.dialect.mysql.visitor.MySqlSchemaStatVisitor;
|
||||
import com.alibaba.druid.sql.visitor.SQLEvalVisitorUtils;
|
||||
import com.alibaba.druid.stat.TableStat;
|
||||
import com.alibaba.druid.stat.TableStat.Column;
|
||||
import com.alibaba.druid.stat.TableStat.Condition;
|
||||
import com.alibaba.druid.stat.TableStat.Relationship;
|
||||
@@ -33,12 +35,12 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
private String notSupportMsg = null;
|
||||
private boolean hasOrCondition = false;
|
||||
private List<WhereUnit> whereUnits = new CopyOnWriteArrayList<>();
|
||||
private List<WhereUnit> storedWhereUnits = new CopyOnWriteArrayList<>();
|
||||
private boolean notInWhere = false;
|
||||
private List<SQLSelect> subQueryList = new ArrayList<>();
|
||||
private Map<String, String> aliasMap = new LinkedHashMap<>();
|
||||
private List<String> selectTableList = new ArrayList<>();
|
||||
private String currentTable;
|
||||
private boolean firstSelectBlock = true;
|
||||
|
||||
private void reset() {
|
||||
this.relationships.clear();
|
||||
@@ -55,10 +57,6 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
return notSupportMsg;
|
||||
}
|
||||
|
||||
public boolean hasOrCondition() {
|
||||
return hasOrCondition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean visit(SQLInSubQueryExpr x) {
|
||||
super.visit(x);
|
||||
@@ -333,12 +331,17 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
|
||||
@Override
|
||||
public boolean visit(SQLSelectQueryBlock x) {
|
||||
if (firstSelectBlock) {
|
||||
firstSelectBlock = false;
|
||||
} else {
|
||||
whereUnits.addAll(getAllWhereUnit());
|
||||
this.relationships.clear();
|
||||
this.conditions.clear();
|
||||
this.hasOrCondition = false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void endVisit(SQLSelectQueryBlock x) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void endVisit(SQLSelect x) {
|
||||
@@ -452,7 +455,7 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
return new Column(table, column);
|
||||
}
|
||||
|
||||
return new Column("UNKNOWN", column);
|
||||
return null;
|
||||
}
|
||||
|
||||
private Column getColumnByExpr(SQLPropertyExpr expr) {
|
||||
@@ -482,9 +485,6 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
/**
|
||||
* get table name of field in between expr
|
||||
*
|
||||
* @param betweenExpr
|
||||
* @param column
|
||||
* @return
|
||||
*/
|
||||
private String getOwnerTableName(SQLBetweenExpr betweenExpr, String column) {
|
||||
if (aliasMap.size() == 1) { //only has 1 table
|
||||
@@ -524,36 +524,12 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
return "";
|
||||
}
|
||||
|
||||
/**
|
||||
* splitConditions
|
||||
*/
|
||||
private List<List<Condition>> splitConditions() {
|
||||
//pre deal with condition and whereUnits
|
||||
reBuildWhereUnits();
|
||||
|
||||
this.storedWhereUnits.addAll(whereUnits);
|
||||
|
||||
//split according to or expr
|
||||
for (WhereUnit whereUnit : whereUnits) {
|
||||
splitUntilNoOr(whereUnit);
|
||||
}
|
||||
loopFindSubOrCondition(storedWhereUnits);
|
||||
|
||||
|
||||
for (WhereUnit whereUnit : storedWhereUnits) {
|
||||
this.resetCondtionsFromWhereUnit(whereUnit);
|
||||
}
|
||||
|
||||
return mergedConditions();
|
||||
}
|
||||
|
||||
/**
|
||||
* Loop all the splitedExprList and try to accept them again
|
||||
* if the Expr in splitedExprList is still a OR-Expr just deal with it
|
||||
* <p>
|
||||
* This function only recursively all child splitedExpr and make them split again
|
||||
*
|
||||
* @param whereUnitList
|
||||
*/
|
||||
private void loopFindSubOrCondition(List<WhereUnit> whereUnitList) {
|
||||
List<WhereUnit> subWhereUnits = new ArrayList<>();
|
||||
@@ -611,7 +587,6 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
* so when there has limit in mtest.id the xtest.id can also get the limit
|
||||
* P.S.:Only order insensitive operator can be optimize,like = or <=>
|
||||
*
|
||||
* @param relationships
|
||||
*/
|
||||
private void relationMerge(Set<Relationship> relationships) {
|
||||
HashSet<Relationship> loopReSet = new HashSet<>();
|
||||
@@ -627,9 +602,7 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
if (tempSet.size() == 2) {
|
||||
Relationship rs1 = new Relationship(tempSet.get(0), tempSet.get(1), inv.getOperator());
|
||||
Relationship rs2 = new Relationship(tempSet.get(1), tempSet.get(0), inv.getOperator());
|
||||
if (relationships.contains(rs1) || relationships.contains(rs2)) {
|
||||
continue;
|
||||
} else {
|
||||
if (!relationships.contains(rs1) && !relationships.contains(rs2)) {
|
||||
relationships.add(rs1);
|
||||
}
|
||||
}
|
||||
@@ -646,8 +619,6 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
* when the result size is 2,we can know that there is 3 columns in 2 relationships
|
||||
* and the derivative relationship may be needed
|
||||
*
|
||||
* @param tempSet
|
||||
* @param tmp
|
||||
*/
|
||||
private void addAndCheckDuplicate(List<Column> tempSet, Column tmp) {
|
||||
if (tempSet.contains(tmp)) {
|
||||
@@ -663,107 +634,6 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
return hasOrCondition;
|
||||
}
|
||||
|
||||
private List<List<Condition>> mergedConditions() {
|
||||
if (storedWhereUnits.size() == 0) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
for (WhereUnit whereUnit : storedWhereUnits) {
|
||||
mergeSubConditionWithOuterCondition(whereUnit);
|
||||
}
|
||||
|
||||
return getMergedConditionList(storedWhereUnits);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* mergeSubConditionWithOuterCondition
|
||||
* Only subWhereUnit will be deal
|
||||
*
|
||||
* @param whereUnit
|
||||
*/
|
||||
private void mergeSubConditionWithOuterCondition(WhereUnit whereUnit) {
|
||||
if (whereUnit.getSubWhereUnit().size() > 0) {
|
||||
for (WhereUnit sub : whereUnit.getSubWhereUnit()) {
|
||||
mergeSubConditionWithOuterCondition(sub);
|
||||
}
|
||||
|
||||
if (whereUnit.getSubWhereUnit().size() > 1) {
|
||||
List<List<Condition>> mergedConditionList = getMergedConditionList(whereUnit.getSubWhereUnit());
|
||||
if (whereUnit.getOutConditions().size() > 0) {
|
||||
for (List<Condition> aMergedConditionList : mergedConditionList) {
|
||||
aMergedConditionList.addAll(whereUnit.getOutConditions());
|
||||
}
|
||||
}
|
||||
if (whereUnit.getOutRelationships().size() > 0) {
|
||||
for (List<Condition> aMergedConditionList : mergedConditionList) {
|
||||
extendConditionsFromRelations(aMergedConditionList, whereUnit.getOutRelationships());
|
||||
}
|
||||
}
|
||||
whereUnit.getConditionList().addAll(mergedConditionList);
|
||||
} else if (whereUnit.getSubWhereUnit().size() == 1) {
|
||||
List<List<Condition>> subConditionList = whereUnit.getSubWhereUnit().get(0).getConditionList();
|
||||
if (whereUnit.getOutConditions().size() > 0 && subConditionList.size() > 0) {
|
||||
for (List<Condition> aSubConditionList : subConditionList) {
|
||||
aSubConditionList.addAll(whereUnit.getOutConditions());
|
||||
}
|
||||
}
|
||||
if (whereUnit.getOutRelationships().size() > 0 && subConditionList.size() > 0) {
|
||||
for (List<Condition> aSubConditionList : subConditionList) {
|
||||
extendConditionsFromRelations(aSubConditionList, whereUnit.getOutRelationships());
|
||||
}
|
||||
}
|
||||
whereUnit.getConditionList().addAll(subConditionList);
|
||||
}
|
||||
} else {
|
||||
//do nothing
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* merge WhereUnit's condition
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
private List<List<Condition>> getMergedConditionList(List<WhereUnit> whereUnitList) {
|
||||
List<List<Condition>> mergedConditionList = new ArrayList<>();
|
||||
if (whereUnitList.size() == 0) {
|
||||
return mergedConditionList;
|
||||
}
|
||||
mergedConditionList.addAll(whereUnitList.get(0).getConditionList());
|
||||
|
||||
for (int i = 1; i < whereUnitList.size(); i++) {
|
||||
mergedConditionList = merge(mergedConditionList, whereUnitList.get(i).getConditionList());
|
||||
}
|
||||
return mergedConditionList;
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge 2 list
|
||||
*
|
||||
* @param list1
|
||||
* @param list2
|
||||
* @return
|
||||
*/
|
||||
private List<List<Condition>> merge(List<List<Condition>> list1, List<List<Condition>> list2) {
|
||||
if (list1.size() == 0) {
|
||||
return list2;
|
||||
} else if (list2.size() == 0) {
|
||||
return list1;
|
||||
}
|
||||
|
||||
List<List<Condition>> retList = new ArrayList<>();
|
||||
for (List<Condition> aList1 : list1) {
|
||||
for (List<Condition> aList2 : list2) {
|
||||
List<Condition> listTmp = new ArrayList<>();
|
||||
listTmp.addAll(aList1);
|
||||
listTmp.addAll(aList2);
|
||||
retList.add(listTmp);
|
||||
}
|
||||
}
|
||||
return retList;
|
||||
}
|
||||
|
||||
public List<String> getSelectTableList() {
|
||||
return selectTableList;
|
||||
}
|
||||
@@ -774,9 +644,8 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
* exp (conditionA OR conditionB) into conditionList{conditionA,conditionB}
|
||||
* so the conditionA,conditionB can be group with outer conditions
|
||||
*
|
||||
* @param whereUnit
|
||||
*/
|
||||
private void resetCondtionsFromWhereUnit(WhereUnit whereUnit) {
|
||||
private void resetConditionsFromWhereUnit(WhereUnit whereUnit) {
|
||||
List<List<Condition>> retList = new ArrayList<>();
|
||||
List<Condition> outSideCondition = new ArrayList<>();
|
||||
outSideCondition.addAll(conditions);
|
||||
@@ -792,22 +661,21 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
Set<Relationship> relations = new HashSet<>();
|
||||
relations.addAll(getRelationships());
|
||||
relations.addAll(outSideRelationship);
|
||||
extendConditionsFromRelations(conds, relations);
|
||||
ConditionUtil.extendConditionsFromRelations(conds, relations);
|
||||
retList.add(conds);
|
||||
this.conditions.clear();
|
||||
this.relationships.clear();
|
||||
}
|
||||
whereUnit.setConditionList(retList);
|
||||
whereUnit.setOrConditionList(retList);
|
||||
|
||||
for (WhereUnit subWhere : whereUnit.getSubWhereUnit()) {
|
||||
resetCondtionsFromWhereUnit(subWhere);
|
||||
resetConditionsFromWhereUnit(subWhere);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* split on conditions into whereUnit..splitedExprList
|
||||
*
|
||||
* @param whereUnit
|
||||
*/
|
||||
private void splitUntilNoOr(WhereUnit whereUnit) {
|
||||
if (whereUnit.isFinishedParse()) {
|
||||
@@ -818,7 +686,9 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
}
|
||||
} else {
|
||||
SQLBinaryOpExpr expr = whereUnit.getCanSplitExpr();
|
||||
if (expr.getOperator() == SQLBinaryOperator.BooleanOr) {
|
||||
if (expr == null) {
|
||||
whereUnit.setFinishedParse(true);
|
||||
} else if (expr.getOperator() == SQLBinaryOperator.BooleanOr) {
|
||||
addExprNotFalse(whereUnit, expr.getRight());
|
||||
if (expr.getLeft() instanceof SQLBinaryOpExpr) {
|
||||
whereUnit.setCanSplitExpr((SQLBinaryOpExpr) expr.getLeft());
|
||||
@@ -871,47 +741,65 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor {
|
||||
return currentTable;
|
||||
}
|
||||
|
||||
private void extendConditionsFromRelations(List<Condition> conds, Set<Relationship> relations) {
|
||||
List<Condition> newConds = new ArrayList<>();
|
||||
Iterator<Condition> iterator = conds.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Condition condition = iterator.next();
|
||||
if (condition.getValues().size() == 0) {
|
||||
iterator.remove();
|
||||
continue;
|
||||
}
|
||||
if (!condition.getOperator().equals("=") && !condition.getOperator().equals("<=>")) {
|
||||
continue;
|
||||
}
|
||||
Column column = condition.getColumn();
|
||||
for (Relationship relation : relations) {
|
||||
if (!condition.getOperator().equalsIgnoreCase(relation.getOperator())) {
|
||||
continue;
|
||||
}
|
||||
if (column.equals(relation.getLeft())) {
|
||||
Condition cond = new Condition(relation.getRight(), condition.getOperator());
|
||||
cond.getValues().addAll(condition.getValues());
|
||||
newConds.add(cond);
|
||||
} else if (column.equals(relation.getRight())) {
|
||||
Condition cond = new Condition(relation.getLeft(), condition.getOperator());
|
||||
cond.getValues().addAll(condition.getValues());
|
||||
newConds.add(cond);
|
||||
private static void mergeOuterRelations(WhereUnit whereUnit) {
|
||||
if (whereUnit.getSubWhereUnit().size() > 0) {
|
||||
for (WhereUnit sub : whereUnit.getSubWhereUnit()) {
|
||||
mergeOuterRelations(sub);
|
||||
if (whereUnit.getOutRelationships().size() > 0) {
|
||||
for (List<TableStat.Condition> subConditionList : sub.getOrConditionList()) {
|
||||
ConditionUtil.extendConditionsFromRelations(subConditionList, whereUnit.getOutRelationships());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
conds.addAll(newConds);
|
||||
}
|
||||
|
||||
public List<List<Condition>> getConditionList() {
|
||||
if (this.hasOrCondition()) {
|
||||
return this.splitConditions();
|
||||
public List<WhereUnit> getAllWhereUnit() {
|
||||
List<WhereUnit> storedWhereUnits = new ArrayList<>();
|
||||
if (this.hasOrCondition) {
|
||||
//pre deal with condition and whereUnits
|
||||
reBuildWhereUnits();
|
||||
storedWhereUnits.addAll(whereUnits);
|
||||
|
||||
//split according to or expr
|
||||
for (WhereUnit whereUnit : whereUnits) {
|
||||
splitUntilNoOr(whereUnit);
|
||||
}
|
||||
whereUnits.clear();
|
||||
loopFindSubOrCondition(storedWhereUnits);
|
||||
|
||||
for (WhereUnit whereUnit : storedWhereUnits) {
|
||||
this.resetConditionsFromWhereUnit(whereUnit);
|
||||
}
|
||||
} else {
|
||||
List<Condition> conds = this.getConditions();
|
||||
Set<Relationship> relations = getRelationships();
|
||||
extendConditionsFromRelations(conds, relations);
|
||||
List<List<Condition>> result = new ArrayList<>();
|
||||
result.add(conds);
|
||||
return result;
|
||||
storedWhereUnits.addAll(whereUnits);
|
||||
whereUnits.clear();
|
||||
WhereUnit whereUnit = generateWhereUnit();
|
||||
if (whereUnit != null) {
|
||||
storedWhereUnits.add(whereUnit);
|
||||
}
|
||||
}
|
||||
|
||||
for (WhereUnit whereUnit : storedWhereUnits) {
|
||||
mergeOuterRelations(whereUnit);
|
||||
}
|
||||
|
||||
return storedWhereUnits;
|
||||
}
|
||||
|
||||
private WhereUnit generateWhereUnit() {
|
||||
List<Condition> conditionList = new ArrayList<>();
|
||||
conditionList.addAll(this.getConditions());
|
||||
ConditionUtil.extendConditionsFromRelations(conditionList, this.relationships);
|
||||
if (conditionList.size() == 0 && this.relationships.size() == 0) {
|
||||
return null;
|
||||
}
|
||||
WhereUnit whereUnit = new WhereUnit();
|
||||
whereUnit.setFinishedParse(true);
|
||||
List<List<Condition>> retList = new ArrayList<>();
|
||||
retList.add(conditionList);
|
||||
whereUnit.setOrConditionList(retList);
|
||||
whereUnit.addOutRelationships(this.relationships);
|
||||
return whereUnit;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -52,11 +52,11 @@ public class WhereUnit {
|
||||
|
||||
private List<SQLExpr> splitedExprList = new ArrayList<>();
|
||||
|
||||
private List<List<Condition>> conditionList = new ArrayList<>();
|
||||
private List<List<Condition>> orConditionList = new ArrayList<>();
|
||||
/**
|
||||
* whereExpris not contains all where condition,consider outConditions
|
||||
*/
|
||||
private List<Condition> outConditions = new ArrayList<>();
|
||||
private List<Condition> outAndConditions = new ArrayList<>();
|
||||
|
||||
private Set<Relationship> outRelationships = new HashSet<>();
|
||||
|
||||
@@ -65,12 +65,12 @@ public class WhereUnit {
|
||||
|
||||
private boolean finishedParse = false;
|
||||
|
||||
public List<Condition> getOutConditions() {
|
||||
return outConditions;
|
||||
public List<Condition> getOutAndConditions() {
|
||||
return outAndConditions;
|
||||
}
|
||||
|
||||
public void addOutConditions(List<Condition> conditions) {
|
||||
this.outConditions.addAll(conditions);
|
||||
this.outAndConditions.addAll(conditions);
|
||||
}
|
||||
|
||||
public Set<Relationship> getOutRelationships() {
|
||||
@@ -113,12 +113,12 @@ public class WhereUnit {
|
||||
this.splitedExprList.add(splitedExpr);
|
||||
}
|
||||
|
||||
public List<List<Condition>> getConditionList() {
|
||||
return conditionList;
|
||||
public List<List<Condition>> getOrConditionList() {
|
||||
return orConditionList;
|
||||
}
|
||||
|
||||
public void setConditionList(List<List<Condition>> conditionList) {
|
||||
this.conditionList = conditionList;
|
||||
public void setOrConditionList(List<List<Condition>> orConditionList) {
|
||||
this.orConditionList = orConditionList;
|
||||
}
|
||||
|
||||
public void addSubWhereUnit(WhereUnit whereUnit) {
|
||||
@@ -128,4 +128,63 @@ public class WhereUnit {
|
||||
public List<WhereUnit> getSubWhereUnit() {
|
||||
return this.subWhereUnits;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (Condition cond : outAndConditions) {
|
||||
if (sb.length() > 0) {
|
||||
sb.append(" and ");
|
||||
}
|
||||
sb.append("(");
|
||||
sb.append(cond);
|
||||
sb.append(")");
|
||||
}
|
||||
if (orConditionList.size() > 0) {
|
||||
if (outAndConditions.size() > 0) {
|
||||
sb.append(" and (");
|
||||
}
|
||||
int iOrCnt = 0;
|
||||
for (List<Condition> or : orConditionList) {
|
||||
if (iOrCnt > 0) {
|
||||
sb.append(" or ");
|
||||
}
|
||||
sb.append("(");
|
||||
int jCnt = 0;
|
||||
for (Condition innerOr : or) {
|
||||
if (jCnt > 0) {
|
||||
sb.append(" and ");
|
||||
}
|
||||
sb.append("(");
|
||||
sb.append(innerOr);
|
||||
sb.append(")");
|
||||
jCnt++;
|
||||
}
|
||||
sb.append(")");
|
||||
iOrCnt++;
|
||||
}
|
||||
if (subWhereUnits.size() > 0) {
|
||||
sb.append(" or ");
|
||||
sb.append("(");
|
||||
for (WhereUnit subWhereUnit : subWhereUnits) {
|
||||
sb.append(subWhereUnit);
|
||||
}
|
||||
sb.append(")");
|
||||
}
|
||||
|
||||
if (outAndConditions.size() > 0) {
|
||||
sb.append(" )");
|
||||
}
|
||||
} else if (subWhereUnits.size() > 0) {
|
||||
if (outAndConditions.size() > 0) {
|
||||
sb.append(" and ");
|
||||
}
|
||||
sb.append("(");
|
||||
for (WhereUnit subWhereUnit : subWhereUnits) {
|
||||
sb.append(subWhereUnit);
|
||||
}
|
||||
sb.append(")");
|
||||
}
|
||||
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -13,18 +13,18 @@ import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.config.model.TableConfig;
|
||||
import com.actiontech.dble.plan.common.ptr.StringPtr;
|
||||
import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.RouteResultsetNode;
|
||||
import com.actiontech.dble.route.parser.druid.DruidParser;
|
||||
import com.actiontech.dble.route.parser.druid.DruidShardingParseInfo;
|
||||
import com.actiontech.dble.route.parser.druid.RouteCalculateUnit;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.route.util.ConditionUtil;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.actiontech.dble.server.ServerConnection;
|
||||
import com.actiontech.dble.sqlengine.mpp.IsValue;
|
||||
import com.actiontech.dble.sqlengine.mpp.RangeValue;
|
||||
import com.actiontech.dble.util.StringUtil;
|
||||
import com.alibaba.druid.sql.ast.SQLStatement;
|
||||
import com.alibaba.druid.sql.dialect.mysql.visitor.MySqlOutputVisitor;
|
||||
import com.alibaba.druid.stat.TableStat.Condition;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@@ -39,7 +39,7 @@ import java.util.*;
|
||||
*/
|
||||
public class DefaultDruidParser implements DruidParser {
|
||||
protected static final Logger LOGGER = LoggerFactory.getLogger(DefaultDruidParser.class);
|
||||
protected DruidShardingParseInfo ctx;
|
||||
DruidShardingParseInfo ctx;
|
||||
|
||||
|
||||
public DefaultDruidParser() {
|
||||
@@ -66,60 +66,71 @@ public class DefaultDruidParser implements DruidParser {
|
||||
}
|
||||
|
||||
@Override
|
||||
public SchemaConfig visitorParse(SchemaConfig schema, RouteResultset rrs, SQLStatement stmt, ServerSchemaStatVisitor visitor, ServerConnection sc)
|
||||
public SchemaConfig visitorParse(SchemaConfig schemaConfig, RouteResultset rrs, SQLStatement stmt, ServerSchemaStatVisitor visitor, ServerConnection sc)
|
||||
throws SQLException {
|
||||
stmt.accept(visitor);
|
||||
if (visitor.getNotSupportMsg() != null) {
|
||||
throw new SQLNonTransientException(visitor.getNotSupportMsg());
|
||||
}
|
||||
Map<String, String> tableAliasMap = getTableAliasMap(visitor.getAliasMap());
|
||||
ctx.setRouteCalculateUnits(this.buildRouteCalculateUnits(tableAliasMap, visitor.getConditionList()));
|
||||
return schema;
|
||||
String schemaName = null;
|
||||
if (schemaConfig != null) {
|
||||
schemaName = schemaConfig.getName();
|
||||
}
|
||||
Map<String, String> tableAliasMap = getTableAliasMap(schemaName, visitor.getAliasMap());
|
||||
ctx.setRouteCalculateUnits(ConditionUtil.buildRouteCalculateUnits(visitor.getAllWhereUnit(), tableAliasMap, schemaName));
|
||||
|
||||
return schemaConfig;
|
||||
}
|
||||
|
||||
private Map<String, String> getTableAliasMap(Map<String, String> originTableAliasMap) {
|
||||
private Map<String, String> getTableAliasMap(String defaultSchemaName, Map<String, String> originTableAliasMap) {
|
||||
if (originTableAliasMap == null) {
|
||||
return null;
|
||||
}
|
||||
Map<String, String> tableAliasMap = new HashMap<>();
|
||||
tableAliasMap.putAll(originTableAliasMap);
|
||||
|
||||
Map<String, String> tableAliasMap = new HashMap<>(originTableAliasMap);
|
||||
for (Map.Entry<String, String> entry : originTableAliasMap.entrySet()) {
|
||||
String key = entry.getKey();
|
||||
String value = entry.getValue();
|
||||
if (DbleServer.getInstance().getSystemVariables().isLowerCaseTableNames()) {
|
||||
if (key != null) {
|
||||
key = key.toLowerCase();
|
||||
}
|
||||
if (value != null) {
|
||||
value = value.toLowerCase();
|
||||
}
|
||||
// fixme: not strict
|
||||
if (key != null && key.startsWith("`")) {
|
||||
tableAliasMap.put(key.replaceAll("`", ""), value);
|
||||
}
|
||||
}
|
||||
|
||||
Iterator<Map.Entry<String, String>> iterator = tableAliasMap.entrySet().iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Map.Entry<String, String> next = iterator.next();
|
||||
String keySchemaName = defaultSchemaName;
|
||||
String valueSchemaName = defaultSchemaName;
|
||||
String key = next.getKey();
|
||||
String value = next.getValue();
|
||||
if ("subquery".equalsIgnoreCase(value)) {
|
||||
iterator.remove();
|
||||
continue;
|
||||
}
|
||||
if (key != null) {
|
||||
int pos = key.indexOf(".");
|
||||
if (pos > 0) {
|
||||
keySchemaName = key.substring(0, pos);
|
||||
key = key.substring(pos + 1);
|
||||
}
|
||||
}
|
||||
if (value != null) {
|
||||
int pos = value.indexOf(".");
|
||||
if (pos > 0) {
|
||||
valueSchemaName = value.substring(0, pos);
|
||||
value = value.substring(pos + 1);
|
||||
}
|
||||
}
|
||||
if (key != null && key.charAt(0) == '`') {
|
||||
key = key.substring(1, key.length() - 1);
|
||||
}
|
||||
if (value != null && value.charAt(0) == '`') {
|
||||
value = value.substring(1, value.length() - 1);
|
||||
}
|
||||
// remove database in database.table
|
||||
if (key != null) {
|
||||
boolean needAddTable = false;
|
||||
if (key.equals(value)) {
|
||||
needAddTable = true;
|
||||
}
|
||||
if (needAddTable) {
|
||||
ctx.addTable(key);
|
||||
if (key != null && keySchemaName != null) {
|
||||
keySchemaName = StringUtil.removeBackQuote(keySchemaName);
|
||||
key = StringUtil.removeBackQuote(key);
|
||||
// remove database in database.table
|
||||
if (key.equals(value) && keySchemaName.equals(valueSchemaName)) {
|
||||
Pair<String, String> tmpTable = new Pair<>(keySchemaName, key);
|
||||
if (!ctx.getTables().contains(tmpTable)) {
|
||||
ctx.addTable(tmpTable);
|
||||
}
|
||||
}
|
||||
tableAliasMap.put(key, value);
|
||||
}
|
||||
@@ -128,58 +139,6 @@ public class DefaultDruidParser implements DruidParser {
|
||||
return tableAliasMap;
|
||||
}
|
||||
|
||||
private List<RouteCalculateUnit> buildRouteCalculateUnits(Map<String, String> tableAliasMap, List<List<Condition>> conditionList) {
|
||||
List<RouteCalculateUnit> retList = new ArrayList<>();
|
||||
//find partition column in condition
|
||||
for (List<Condition> aConditionList : conditionList) {
|
||||
RouteCalculateUnit routeCalculateUnit = new RouteCalculateUnit();
|
||||
for (Condition condition : aConditionList) {
|
||||
List<Object> values = condition.getValues();
|
||||
if (values.size() == 0) {
|
||||
continue;
|
||||
}
|
||||
if (checkConditionValues(values)) {
|
||||
String columnName = StringUtil.removeBackQuote(condition.getColumn().getName().toUpperCase());
|
||||
String tableName = StringUtil.removeBackQuote(condition.getColumn().getTable());
|
||||
if (DbleServer.getInstance().getSystemVariables().isLowerCaseTableNames()) {
|
||||
tableName = tableName.toLowerCase();
|
||||
}
|
||||
if (tableAliasMap != null && tableAliasMap.get(tableName) == null) {
|
||||
//ignore subQuery's alias
|
||||
continue;
|
||||
}
|
||||
if (tableAliasMap != null && tableAliasMap.get(tableName) != null &&
|
||||
!tableAliasMap.get(tableName).equals(tableName)) {
|
||||
tableName = tableAliasMap.get(tableName);
|
||||
}
|
||||
String operator = condition.getOperator();
|
||||
|
||||
//execute only between ,in and =
|
||||
if (operator.equals("between")) {
|
||||
RangeValue rv = new RangeValue(values.get(0), values.get(1), RangeValue.EE);
|
||||
routeCalculateUnit.addShardingExpr(tableName, columnName, rv);
|
||||
} else if (operator.equals("=") || operator.toLowerCase().equals("in")) {
|
||||
routeCalculateUnit.addShardingExpr(tableName, columnName, values.toArray());
|
||||
} else if (operator.equals("IS")) {
|
||||
IsValue isValue = new IsValue(values.toArray());
|
||||
routeCalculateUnit.addShardingExpr(tableName, columnName, isValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
retList.add(routeCalculateUnit);
|
||||
}
|
||||
return retList;
|
||||
}
|
||||
|
||||
private boolean checkConditionValues(List<Object> values) {
|
||||
for (Object value : values) {
|
||||
if (value != null) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public DruidShardingParseInfo getCtx() {
|
||||
return ctx;
|
||||
}
|
||||
@@ -229,4 +188,32 @@ public class DefaultDruidParser implements DruidParser {
|
||||
statement.accept(visitor);
|
||||
return buf.toString();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* delete / update sharding table with limit route
|
||||
* if the update/delete with limit route to more than one sharding-table throw a new Execption
|
||||
*
|
||||
*/
|
||||
void updateAndDeleteLimitRoute(RouteResultset rrs, String tableName, SchemaConfig schema) throws SQLException {
|
||||
SortedSet<RouteResultsetNode> nodeSet = new TreeSet<>();
|
||||
for (RouteCalculateUnit unit : ctx.getRouteCalculateUnits()) {
|
||||
RouteResultset rrsTmp = RouterUtil.tryRouteForOneTable(schema, unit, tableName, rrs, false, DbleServer.getInstance().getRouterService().getTableId2DataNodeCache(), null);
|
||||
if (rrsTmp != null && rrsTmp.getNodes() != null) {
|
||||
Collections.addAll(nodeSet, rrsTmp.getNodes());
|
||||
}
|
||||
}
|
||||
if (nodeSet.size() > 1) {
|
||||
throw new SQLNonTransientException("delete/update sharding table with a limit route to multiNode not support");
|
||||
} else {
|
||||
RouteResultsetNode[] nodes = new RouteResultsetNode[nodeSet.size()];
|
||||
int i = 0;
|
||||
for (RouteResultsetNode aNodeSet : nodeSet) {
|
||||
nodes[i] = aNodeSet;
|
||||
i++;
|
||||
}
|
||||
rrs.setNodes(nodes);
|
||||
rrs.setFinishedRoute(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -6,8 +6,6 @@
|
||||
package com.actiontech.dble.route.parser.druid.impl;
|
||||
|
||||
import com.actiontech.dble.DbleServer;
|
||||
import com.actiontech.dble.backend.mysql.nio.handler.FetchStoreNodeOfChildTableHandler;
|
||||
import com.actiontech.dble.config.ErrorCode;
|
||||
import com.actiontech.dble.config.ServerPrivileges;
|
||||
import com.actiontech.dble.config.ServerPrivileges.CheckType;
|
||||
import com.actiontech.dble.config.model.SchemaConfig;
|
||||
@@ -18,6 +16,7 @@ import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.RouteResultsetNode;
|
||||
import com.actiontech.dble.route.function.AbstractPartitionAlgorithm;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.actiontech.dble.server.ServerConnection;
|
||||
import com.actiontech.dble.server.util.GlobalTableUtil;
|
||||
@@ -107,7 +106,7 @@ public class DruidInsertParser extends DruidInsertReplaceParser {
|
||||
}
|
||||
} else {
|
||||
rrs.setStatement(RouterUtil.removeSchema(rrs.getStatement(), schemaInfo.getSchema()));
|
||||
ctx.addTable(tableName);
|
||||
ctx.addTable(new Pair<>(schema.getName(), tableName));
|
||||
}
|
||||
return schema;
|
||||
}
|
||||
@@ -163,28 +162,7 @@ public class DruidInsertParser extends DruidInsertReplaceParser {
|
||||
rrs.setFinishedRoute(true);
|
||||
} else {
|
||||
rrs.setFinishedExecute(true);
|
||||
DbleServer.getInstance().getComplexQueryExecutor().execute(new Runnable() {
|
||||
//get child result will be blocked, so use ComplexQueryExecutor
|
||||
@Override
|
||||
public void run() {
|
||||
// route by sql query root parent's data node
|
||||
String findRootTBSql = tc.getLocateRTableKeySql() + joinKeyVal;
|
||||
if (LOGGER.isDebugEnabled()) {
|
||||
LOGGER.debug("to find root parent's node sql :" + findRootTBSql);
|
||||
}
|
||||
FetchStoreNodeOfChildTableHandler fetchHandler = new FetchStoreNodeOfChildTableHandler(findRootTBSql, sc.getSession2());
|
||||
String dn = fetchHandler.execute(schema.getName(), tc.getRootParent().getDataNodes());
|
||||
if (dn == null) {
|
||||
sc.writeErrMessage(ErrorCode.ER_UNKNOWN_ERROR, "can't find (root) parent sharding node for sql:" + sql);
|
||||
return;
|
||||
}
|
||||
if (LOGGER.isDebugEnabled()) {
|
||||
LOGGER.debug("found partition node for child table to insert " + dn + " sql :" + sql);
|
||||
}
|
||||
RouterUtil.routeToSingleNode(rrs, dn);
|
||||
sc.getSession2().execute(rrs);
|
||||
}
|
||||
});
|
||||
fetchChildTableToRoute(tc, joinKeyVal, sc, schema, sql, rrs);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -272,9 +250,7 @@ public class DruidInsertParser extends DruidInsertReplaceParser {
|
||||
LOGGER.info(msg);
|
||||
throw new SQLNonTransientException(msg);
|
||||
}
|
||||
if (nodeValuesMap.get(nodeIndex) == null) {
|
||||
nodeValuesMap.put(nodeIndex, new ArrayList<ValuesClause>());
|
||||
}
|
||||
nodeValuesMap.putIfAbsent(nodeIndex, new ArrayList<>());
|
||||
nodeValuesMap.get(nodeIndex).add(valueClause);
|
||||
}
|
||||
|
||||
|
||||
@@ -7,6 +7,11 @@ package com.actiontech.dble.route.parser.druid.impl;
|
||||
|
||||
import com.actiontech.dble.DbleServer;
|
||||
import com.actiontech.dble.config.model.TableConfig;
|
||||
import com.actiontech.dble.backend.mysql.nio.handler.FetchStoreNodeOfChildTableHandler;
|
||||
import com.actiontech.dble.config.ErrorCode;
|
||||
import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.net.ConnectionException;
|
||||
import com.actiontech.dble.server.ServerConnection;
|
||||
import com.actiontech.dble.meta.protocol.StructureMeta;
|
||||
import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
@@ -27,7 +32,7 @@ import java.util.Set;
|
||||
import static com.actiontech.dble.server.util.SchemaUtil.SchemaInfo;
|
||||
|
||||
abstract class DruidInsertReplaceParser extends DefaultDruidParser {
|
||||
protected static RouteResultset routeByERParentKey(RouteResultset rrs, TableConfig tc, String joinKeyVal, SchemaInfo schemaInfo)
|
||||
static RouteResultset routeByERParentKey(RouteResultset rrs, TableConfig tc, String joinKeyVal, SchemaInfo schemaInfo)
|
||||
throws SQLNonTransientException {
|
||||
if (tc.getDirectRouteTC() != null) {
|
||||
Set<ColumnRoutePair> parentColVal = new HashSet<>(1);
|
||||
@@ -51,10 +56,8 @@ abstract class DruidInsertReplaceParser extends DefaultDruidParser {
|
||||
/**
|
||||
* check if the column is not null and the
|
||||
*
|
||||
* @param columnValue
|
||||
* @throws SQLNonTransientException
|
||||
*/
|
||||
public static void checkDefaultValues(String columnValue, TableConfig tableConfig, String schema, String partitionColumn) throws SQLNonTransientException {
|
||||
static void checkDefaultValues(String columnValue, TableConfig tableConfig, String schema, String partitionColumn) throws SQLNonTransientException {
|
||||
|
||||
if (columnValue == null || "null".equalsIgnoreCase(columnValue)) {
|
||||
StructureMeta.TableMeta meta = DbleServer.getInstance().getTmManager().getSyncTableMeta(schema, tableConfig.getName());
|
||||
@@ -70,7 +73,7 @@ abstract class DruidInsertReplaceParser extends DefaultDruidParser {
|
||||
}
|
||||
}
|
||||
|
||||
protected static String shardingValueToSting(SQLExpr valueExpr) throws SQLNonTransientException {
|
||||
static String shardingValueToSting(SQLExpr valueExpr) throws SQLNonTransientException {
|
||||
String shardingValue = null;
|
||||
if (valueExpr instanceof SQLIntegerExpr) {
|
||||
SQLIntegerExpr intExpr = (SQLIntegerExpr) valueExpr;
|
||||
@@ -103,7 +106,7 @@ abstract class DruidInsertReplaceParser extends DefaultDruidParser {
|
||||
return idxGlobal;
|
||||
}
|
||||
|
||||
protected int getIncrementKeyIndex(SchemaInfo schemaInfo, String incrementColumn) throws SQLNonTransientException {
|
||||
int getIncrementKeyIndex(SchemaInfo schemaInfo, String incrementColumn) throws SQLNonTransientException {
|
||||
if (incrementColumn == null) {
|
||||
throw new SQLNonTransientException("please make sure the primaryKey's config is not null in schemal.xml");
|
||||
}
|
||||
@@ -136,7 +139,7 @@ abstract class DruidInsertReplaceParser extends DefaultDruidParser {
|
||||
return primaryKeyIndex;
|
||||
}
|
||||
|
||||
protected int getTableColumns(SchemaInfo schemaInfo, List<SQLExpr> columnExprList)
|
||||
int getTableColumns(SchemaInfo schemaInfo, List<SQLExpr> columnExprList)
|
||||
throws SQLNonTransientException {
|
||||
if (columnExprList == null || columnExprList.size() == 0) {
|
||||
StructureMeta.TableMeta tbMeta = DbleServer.getInstance().getTmManager().getSyncTableMeta(schemaInfo.getSchema(), schemaInfo.getTable());
|
||||
@@ -151,7 +154,7 @@ abstract class DruidInsertReplaceParser extends DefaultDruidParser {
|
||||
}
|
||||
}
|
||||
|
||||
protected int getShardingColIndex(SchemaInfo schemaInfo, List<SQLExpr> columnExprList, String partitionColumn) throws SQLNonTransientException {
|
||||
int getShardingColIndex(SchemaInfo schemaInfo, List<SQLExpr> columnExprList, String partitionColumn) throws SQLNonTransientException {
|
||||
int shardingColIndex = -1;
|
||||
if (columnExprList == null || columnExprList.size() == 0) {
|
||||
StructureMeta.TableMeta tbMeta = DbleServer.getInstance().getTmManager().getSyncTableMeta(schemaInfo.getSchema(), schemaInfo.getTable());
|
||||
@@ -172,4 +175,34 @@ abstract class DruidInsertReplaceParser extends DefaultDruidParser {
|
||||
return shardingColIndex;
|
||||
}
|
||||
|
||||
|
||||
void fetchChildTableToRoute(TableConfig tc, String joinKeyVal, ServerConnection sc, SchemaConfig schema, String sql, RouteResultset rrs) {
|
||||
DbleServer.getInstance().getComplexQueryExecutor().execute(new Runnable() {
|
||||
//get child result will be blocked, so use ComplexQueryExecutor
|
||||
@Override
|
||||
public void run() {
|
||||
// route by sql query root parent's data node
|
||||
String findRootTBSql = tc.getLocateRTableKeySql() + joinKeyVal;
|
||||
if (LOGGER.isDebugEnabled()) {
|
||||
LOGGER.debug("to find root parent's node sql :" + findRootTBSql);
|
||||
}
|
||||
FetchStoreNodeOfChildTableHandler fetchHandler = new FetchStoreNodeOfChildTableHandler(findRootTBSql, sc.getSession2());
|
||||
try {
|
||||
String dn = fetchHandler.execute(schema.getName(), tc.getRootParent().getDataNodes());
|
||||
if (dn == null) {
|
||||
sc.writeErrMessage(ErrorCode.ER_UNKNOWN_ERROR, "can't find (root) parent sharding node for sql:" + sql);
|
||||
return;
|
||||
}
|
||||
if (LOGGER.isDebugEnabled()) {
|
||||
LOGGER.debug("found partition node for child table to insert " + dn + " sql :" + sql);
|
||||
}
|
||||
RouterUtil.routeToSingleNode(rrs, dn);
|
||||
sc.getSession2().execute(rrs);
|
||||
} catch (ConnectionException e) {
|
||||
sc.setTxInterrupt(e.toString());
|
||||
sc.writeErrMessage(ErrorCode.ER_UNKNOWN_ERROR, e.toString());
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@@ -6,8 +6,6 @@
|
||||
package com.actiontech.dble.route.parser.druid.impl;
|
||||
|
||||
import com.actiontech.dble.DbleServer;
|
||||
import com.actiontech.dble.backend.mysql.nio.handler.FetchStoreNodeOfChildTableHandler;
|
||||
import com.actiontech.dble.config.ErrorCode;
|
||||
import com.actiontech.dble.config.ServerPrivileges;
|
||||
import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.config.model.TableConfig;
|
||||
@@ -17,6 +15,7 @@ import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.RouteResultsetNode;
|
||||
import com.actiontech.dble.route.function.AbstractPartitionAlgorithm;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.actiontech.dble.server.ServerConnection;
|
||||
import com.actiontech.dble.server.util.GlobalTableUtil;
|
||||
@@ -26,11 +25,7 @@ import com.actiontech.dble.util.StringUtil;
|
||||
import com.alibaba.druid.sql.SQLUtils;
|
||||
import com.alibaba.druid.sql.ast.SQLExpr;
|
||||
import com.alibaba.druid.sql.ast.SQLStatement;
|
||||
import com.alibaba.druid.sql.ast.statement.SQLExprTableSource;
|
||||
import com.alibaba.druid.sql.ast.statement.SQLInsertStatement;
|
||||
import com.alibaba.druid.sql.ast.statement.SQLSelect;
|
||||
import com.alibaba.druid.sql.ast.statement.SQLSelectStatement;
|
||||
import com.alibaba.druid.sql.ast.statement.SQLReplaceStatement;
|
||||
import com.alibaba.druid.sql.ast.statement.*;
|
||||
import com.alibaba.druid.sql.dialect.mysql.parser.MySqlStatementParser;
|
||||
import com.alibaba.druid.sql.parser.SQLStatementParser;
|
||||
|
||||
@@ -112,7 +107,7 @@ public class DruidReplaceParser extends DruidInsertReplaceParser {
|
||||
}
|
||||
} else {
|
||||
rrs.setStatement(RouterUtil.removeSchema(rrs.getStatement(), schemaInfo.getSchema()));
|
||||
ctx.addTable(tableName);
|
||||
ctx.addTable(new Pair<>(schema.getName(), tableName));
|
||||
}
|
||||
|
||||
return schema;
|
||||
@@ -122,13 +117,6 @@ public class DruidReplaceParser extends DruidInsertReplaceParser {
|
||||
/**
|
||||
* check if the nosharding tables are Involved
|
||||
*
|
||||
* @param sc
|
||||
* @param contextSchema
|
||||
* @param schemaInfo
|
||||
* @param rrs
|
||||
* @param replace
|
||||
* @return
|
||||
* @throws SQLException
|
||||
*/
|
||||
private boolean parserNoSharding(ServerConnection sc, String contextSchema, SchemaInfo schemaInfo, RouteResultset rrs, SQLReplaceStatement replace) throws SQLException {
|
||||
String noShardingNode = RouterUtil.isNoSharding(schemaInfo.getSchemaConfig(), schemaInfo.getTable());
|
||||
@@ -238,18 +226,6 @@ public class DruidReplaceParser extends DruidInsertReplaceParser {
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* because of the replace can use a
|
||||
*
|
||||
* @param tableKey
|
||||
* @param values
|
||||
* @param sb
|
||||
* @param autoIncrement
|
||||
* @param idxGlobal
|
||||
* @param colSize
|
||||
* @return
|
||||
* @throws SQLNonTransientException
|
||||
*/
|
||||
private static StringBuilder appendValues(String tableKey, List<SQLExpr> values, StringBuilder sb,
|
||||
int autoIncrement, int idxGlobal, int colSize) throws SQLNonTransientException {
|
||||
// check the value number & the column number is all right
|
||||
@@ -315,31 +291,9 @@ public class DruidReplaceParser extends DruidInsertReplaceParser {
|
||||
rrs.setFinishedRoute(true);
|
||||
} else {
|
||||
rrs.setFinishedExecute(true);
|
||||
DbleServer.getInstance().getComplexQueryExecutor().execute(new Runnable() {
|
||||
//get child result will be blocked, so use ComplexQueryExecutor
|
||||
@Override
|
||||
public void run() {
|
||||
// route by sql query root parent's data node
|
||||
String findRootTBSql = tc.getLocateRTableKeySql() + joinKeyVal;
|
||||
if (LOGGER.isDebugEnabled()) {
|
||||
LOGGER.debug("to find root parent's node sql :" + findRootTBSql);
|
||||
}
|
||||
FetchStoreNodeOfChildTableHandler fetchHandler = new FetchStoreNodeOfChildTableHandler(findRootTBSql, sc.getSession2());
|
||||
String dn = fetchHandler.execute(schema.getName(), tc.getRootParent().getDataNodes());
|
||||
if (dn == null) {
|
||||
sc.writeErrMessage(ErrorCode.ER_UNKNOWN_ERROR, "can't find (root) parent sharding node for sql:" + sql);
|
||||
return;
|
||||
}
|
||||
if (LOGGER.isDebugEnabled()) {
|
||||
LOGGER.debug("found partition node for child table to insert " + dn + " sql :" + sql);
|
||||
}
|
||||
RouterUtil.routeToSingleNode(rrs, dn);
|
||||
sc.getSession2().execute(rrs);
|
||||
}
|
||||
});
|
||||
fetchChildTableToRoute(tc, joinKeyVal, sc, schema, sql, rrs);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isMultiReplace(SQLReplaceStatement insertStmt) {
|
||||
return (insertStmt.getValuesList() != null && insertStmt.getValuesList().size() > 1);
|
||||
}
|
||||
@@ -409,9 +363,7 @@ public class DruidReplaceParser extends DruidInsertReplaceParser {
|
||||
LOGGER.info(msg);
|
||||
throw new SQLNonTransientException(msg);
|
||||
}
|
||||
if (nodeValuesMap.get(nodeIndex) == null) {
|
||||
nodeValuesMap.put(nodeIndex, new ArrayList<SQLInsertStatement.ValuesClause>());
|
||||
}
|
||||
nodeValuesMap.computeIfAbsent(nodeIndex, k -> new ArrayList<>());
|
||||
nodeValuesMap.get(nodeIndex).add(valueClause);
|
||||
}
|
||||
|
||||
|
||||
@@ -21,6 +21,7 @@ import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.RouteResultsetNode;
|
||||
import com.actiontech.dble.route.parser.druid.RouteCalculateUnit;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.actiontech.dble.server.ServerConnection;
|
||||
import com.actiontech.dble.server.handler.MysqlSystemSchemaHandler;
|
||||
@@ -131,7 +132,6 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
if (noShardingNode != null) {
|
||||
//route to singleNode
|
||||
RouterUtil.routeToSingleNode(rrs, noShardingNode);
|
||||
return;
|
||||
} else {
|
||||
//route for configured table
|
||||
TableConfig tc = schema.getTables().get(schemaInfo.getTable());
|
||||
@@ -152,6 +152,7 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (nodeSet.size() == 0) {
|
||||
String msg = " find no Route:" + rrs.getStatement();
|
||||
LOGGER.info(msg);
|
||||
@@ -198,10 +199,9 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
RouterUtil.routeNoNameTableToSingleNode(rrs, schema);
|
||||
}
|
||||
|
||||
|
||||
private SchemaConfig tryRouteToOneNode(SchemaConfig schema, RouteResultset rrs, ServerConnection sc, SQLSelectStatement selectStmt, int tableSize) throws SQLException {
|
||||
private void tryRouteToOneNode(RouteResultset rrs, SQLSelectStatement selectStmt, int tableSize) throws SQLException {
|
||||
Set<String> schemaList = new HashSet<>();
|
||||
String dataNode = RouterUtil.tryRouteTablesToOneNode(sc.getUser(), rrs, schema, ctx, schemaList, tableSize, true);
|
||||
String dataNode = RouterUtil.tryRouteTablesToOneNode(rrs, ctx, schemaList, tableSize, true);
|
||||
if (dataNode != null) {
|
||||
String sql = rrs.getStatement();
|
||||
for (String toRemoveSchemaName : schemaList) {
|
||||
@@ -213,7 +213,6 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
rrs.setNeedOptimizer(true);
|
||||
rrs.setSqlStatement(selectStmt);
|
||||
}
|
||||
return schema;
|
||||
}
|
||||
|
||||
|
||||
@@ -483,7 +482,8 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
rrs.setFinishedExecute(true);
|
||||
return schema;
|
||||
} else {
|
||||
return tryRouteToOneNode(schema, rrs, sc, selectStmt, tableSize);
|
||||
tryRouteToOneNode(rrs, selectStmt, tableSize);
|
||||
return schema;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -505,14 +505,14 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
int limitStart = 0;
|
||||
int limitSize = sqlSchema.getDefaultMaxLimit();
|
||||
|
||||
Map<String, Map<String, Set<ColumnRoutePair>>> allConditions = getAllConditions();
|
||||
Map<Pair<String, String>, Map<String, Set<ColumnRoutePair>>> allConditions = getAllConditions();
|
||||
boolean isNeedAddLimit = isNeedAddLimit(sqlSchema, rrs, mysqlSelectQuery, allConditions);
|
||||
if (isNeedAddLimit) {
|
||||
SQLLimit limit = new SQLLimit();
|
||||
limit.setRowCount(new SQLIntegerExpr(limitSize));
|
||||
mysqlSelectQuery.setLimit(limit);
|
||||
rrs.setLimitSize(limitSize);
|
||||
String sql = getSql(rrs, stmt, isNeedAddLimit, sqlSchema.getName());
|
||||
String sql = RouterUtil.removeSchema(statementToString(stmt), sqlSchema.getName());
|
||||
rrs.changeNodeSqlAfterAddLimit(sql, 0, limitSize);
|
||||
}
|
||||
SQLLimit limit = mysqlSelectQuery.getLimit();
|
||||
@@ -545,102 +545,53 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
}
|
||||
|
||||
mysqlSelectQuery.setLimit(changedLimit);
|
||||
String sql = getSql(rrs, stmt, isNeedAddLimit, sqlSchema.getName());
|
||||
String sql = RouterUtil.removeSchema(statementToString(stmt), sqlSchema.getName());
|
||||
rrs.changeNodeSqlAfterAddLimit(sql, 0, limitStart + limitSize);
|
||||
} else {
|
||||
rrs.changeNodeSqlAfterAddLimit(rrs.getStatement(), rrs.getLimitStart(), rrs.getLimitSize());
|
||||
}
|
||||
}
|
||||
rrs.setCacheAble(isNeedCache(sqlSchema));
|
||||
rrs.setSqlRouteCacheAble(isNeedSqlRouteCache(sqlSchema));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void tryRouteSingleTable(SchemaConfig schema, RouteResultset rrs, LayerCachePool cachePool)
|
||||
throws SQLException {
|
||||
if (rrs.isFinishedRoute()) {
|
||||
return;
|
||||
}
|
||||
SortedSet<RouteResultsetNode> nodeSet = new TreeSet<>();
|
||||
String table = ctx.getTables().get(0);
|
||||
String noShardingNode = RouterUtil.isNoSharding(schema, table);
|
||||
if (noShardingNode != null) {
|
||||
RouterUtil.routeToSingleNode(rrs, noShardingNode);
|
||||
return;
|
||||
}
|
||||
for (RouteCalculateUnit unit : ctx.getRouteCalculateUnits()) {
|
||||
RouteResultset rrsTmp = RouterUtil.tryRouteForOneTable(schema, unit, table, rrs, true, cachePool, null);
|
||||
if (rrsTmp != null && rrsTmp.getNodes() != null) {
|
||||
Collections.addAll(nodeSet, rrsTmp.getNodes());
|
||||
if (rrsTmp.isGlobalTable()) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (nodeSet.size() == 0) {
|
||||
String msg = " find no Route:" + rrs.getStatement();
|
||||
LOGGER.info(msg);
|
||||
throw new SQLNonTransientException(msg);
|
||||
}
|
||||
|
||||
RouteResultsetNode[] nodes = new RouteResultsetNode[nodeSet.size()];
|
||||
int i = 0;
|
||||
for (RouteResultsetNode aNodeSet : nodeSet) {
|
||||
nodes[i] = aNodeSet;
|
||||
i++;
|
||||
}
|
||||
|
||||
rrs.setNodes(nodes);
|
||||
rrs.setFinishedRoute(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* getAllConditions
|
||||
*/
|
||||
private Map<String, Map<String, Set<ColumnRoutePair>>> getAllConditions() {
|
||||
Map<String, Map<String, Set<ColumnRoutePair>>> map = new HashMap<>();
|
||||
private Map<Pair<String, String>, Map<String, Set<ColumnRoutePair>>> getAllConditions() {
|
||||
Map<Pair<String, String>, Map<String, Set<ColumnRoutePair>>> map = new HashMap<>();
|
||||
for (RouteCalculateUnit unit : ctx.getRouteCalculateUnits()) {
|
||||
if (unit != null && unit.getTablesAndConditions() != null) {
|
||||
map.putAll(unit.getTablesAndConditions());
|
||||
}
|
||||
}
|
||||
|
||||
return map;
|
||||
}
|
||||
|
||||
|
||||
protected String getSql(RouteResultset rrs, SQLStatement stmt, boolean isNeedAddLimit, String schema) {
|
||||
if ((isNeedChangeLimit(rrs) || isNeedAddLimit)) {
|
||||
return RouterUtil.removeSchema(statementToString(stmt), schema);
|
||||
}
|
||||
return rrs.getStatement();
|
||||
}
|
||||
|
||||
|
||||
private boolean isNeedChangeLimit(RouteResultset rrs) {
|
||||
if (rrs.getNodes() == null) {
|
||||
return false;
|
||||
} else {
|
||||
return rrs.getNodes().length > 1;
|
||||
}
|
||||
return rrs.getNodes() != null && rrs.getNodes().length > 1;
|
||||
}
|
||||
|
||||
private boolean isNeedCache(SchemaConfig schema) {
|
||||
private boolean isNeedSqlRouteCache(SchemaConfig schema) {
|
||||
if (ctx.getTables() == null || ctx.getTables().size() == 0) {
|
||||
return false;
|
||||
}
|
||||
TableConfig tc = schema.getTables().get(ctx.getTables().get(0));
|
||||
Pair<String, String> table = ctx.getTables().get(0);
|
||||
String tableName = table.getValue();
|
||||
TableConfig tc = schema.getTables().get(tableName);
|
||||
if (tc == null || (ctx.getTables().size() == 1 && tc.isGlobalTable())) {
|
||||
return false;
|
||||
} else {
|
||||
//single table
|
||||
if (ctx.getTables().size() == 1) {
|
||||
String tableName = ctx.getTables().get(0);
|
||||
String primaryKey = schema.getTables().get(tableName).getPrimaryKey();
|
||||
if (ctx.getRouteCalculateUnit().getTablesAndConditions().get(tableName) != null &&
|
||||
ctx.getRouteCalculateUnit().getTablesAndConditions().get(tableName).get(primaryKey) != null &&
|
||||
tc.getDataNodes().size() > 1) { //primaryKey condition
|
||||
return false;
|
||||
String cacheKey = schema.getTables().get(tableName).getPrimaryKey();
|
||||
for (RouteCalculateUnit unit : ctx.getRouteCalculateUnits()) {
|
||||
if (unit.getTablesAndConditions().get(table) != null &&
|
||||
unit.getTablesAndConditions().get(table).get(cacheKey) != null &&
|
||||
tc.getDataNodes().size() > 1) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
return true;
|
||||
@@ -663,15 +614,8 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
mysqlSelectQuery.setLimit(limit);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param schema
|
||||
* @param rrs
|
||||
* @param mysqlSelectQuery
|
||||
* @param allConditions
|
||||
* @return
|
||||
*/
|
||||
private boolean isNeedAddLimit(SchemaConfig schema, RouteResultset rrs,
|
||||
MySqlSelectQueryBlock mysqlSelectQuery, Map<String, Map<String, Set<ColumnRoutePair>>> allConditions) {
|
||||
MySqlSelectQueryBlock mysqlSelectQuery, Map<Pair<String, String>, Map<String, Set<ColumnRoutePair>>> allConditions) {
|
||||
if (rrs.getLimitSize() > -1) {
|
||||
return false;
|
||||
} else if (schema.getDefaultMaxLimit() == -1) {
|
||||
@@ -683,7 +627,8 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
// single table and has primary key , need not limit because of only one row
|
||||
return false;
|
||||
}
|
||||
String tableName = ctx.getTables().get(0);
|
||||
Pair<String, String> table = ctx.getTables().get(0);
|
||||
String tableName = table.getValue();
|
||||
TableConfig tableConfig = schema.getTables().get(tableName);
|
||||
if (tableConfig == null) {
|
||||
return schema.getDefaultMaxLimit() > -1; // get schema's configure
|
||||
@@ -700,7 +645,7 @@ public class DruidSelectParser extends DefaultDruidParser {
|
||||
|
||||
String primaryKey = schema.getTables().get(tableName).getPrimaryKey();
|
||||
// no condition
|
||||
return allConditions.get(tableName) == null || allConditions.get(tableName).get(primaryKey) == null;
|
||||
return allConditions.get(table) == null || allConditions.get(table).get(primaryKey) == null;
|
||||
} else { // no table or multi-table
|
||||
return false;
|
||||
}
|
||||
|
||||
@@ -9,13 +9,13 @@ import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.plan.common.ptr.StringPtr;
|
||||
import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.actiontech.dble.server.ServerConnection;
|
||||
import com.actiontech.dble.server.util.SchemaUtil;
|
||||
import com.alibaba.druid.sql.ast.SQLStatement;
|
||||
import com.alibaba.druid.sql.ast.statement.*;
|
||||
import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlSelectQueryBlock;
|
||||
import com.alibaba.druid.sql.ast.statement.SQLUnionQuery;
|
||||
|
||||
import java.sql.SQLException;
|
||||
import java.util.HashSet;
|
||||
@@ -24,7 +24,7 @@ import java.util.Set;
|
||||
|
||||
public class DruidSingleUnitSelectParser extends DefaultDruidParser {
|
||||
|
||||
private Map<String, SchemaConfig> schemaMap = null;
|
||||
private Map<Pair<String, String>, SchemaConfig> schemaMap = null;
|
||||
|
||||
@Override
|
||||
public SchemaConfig visitorParse(SchemaConfig schema, RouteResultset rrs, SQLStatement stmt,
|
||||
@@ -54,10 +54,9 @@ public class DruidSingleUnitSelectParser extends DefaultDruidParser {
|
||||
for (SchemaConfig schemaInfo : schemaMap.values()) {
|
||||
rrs.setStatement(RouterUtil.removeSchema(rrs.getStatement(), schemaInfo.getName()));
|
||||
}
|
||||
|
||||
super.visitorParse(null, rrs, stmt, visitor, sc);
|
||||
super.visitorParse(schema, rrs, stmt, visitor, sc);
|
||||
if (visitor.getSubQueryList().size() > 0) {
|
||||
this.getCtx().getRouteCalculateUnits().clear();
|
||||
this.getCtx().clearRouteCalculateUnit();
|
||||
}
|
||||
// change canRunInReadDB
|
||||
if ((mysqlSelectQuery.isForUpdate() || mysqlSelectQuery.isLockInShareMode())) {
|
||||
@@ -77,11 +76,11 @@ public class DruidSingleUnitSelectParser extends DefaultDruidParser {
|
||||
}
|
||||
|
||||
|
||||
public Map<String, SchemaConfig> getSchemaMap() {
|
||||
public Map<Pair<String, String>, SchemaConfig> getSchemaMap() {
|
||||
return schemaMap;
|
||||
}
|
||||
|
||||
public void setSchemaMap(Map<String, SchemaConfig> schemaMap) {
|
||||
public void setSchemaMap(Map<Pair<String, String>, SchemaConfig> schemaMap) {
|
||||
this.schemaMap = schemaMap;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -14,6 +14,7 @@ import com.actiontech.dble.meta.protocol.StructureMeta;
|
||||
import com.actiontech.dble.plan.common.ptr.StringPtr;
|
||||
import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.actiontech.dble.server.ServerConnection;
|
||||
import com.actiontech.dble.server.util.GlobalTableUtil;
|
||||
@@ -97,11 +98,11 @@ public class DruidUpdateParser extends DefaultDruidParser {
|
||||
|
||||
confirmChildColumnNotUpdated(update, schema, tableName);
|
||||
|
||||
if (schema.getTables().get(tableName).isGlobalTable() && ctx.getRouteCalculateUnit().getTablesAndConditions().size() > 1) {
|
||||
if (schema.getTables().get(tableName).isGlobalTable() && ctx.getTables().size() > 1) {
|
||||
throw new SQLNonTransientException("global table is not supported in multi table related update " + tableName);
|
||||
}
|
||||
if (ctx.getTables().size() == 0) {
|
||||
ctx.addTable(schemaInfo.getTable());
|
||||
ctx.addTable(new Pair<>(schema.getName(), tableName));
|
||||
}
|
||||
}
|
||||
return schema;
|
||||
@@ -261,7 +262,7 @@ public class DruidUpdateParser extends DefaultDruidParser {
|
||||
LOGGER.info(msg);
|
||||
throw new SQLNonTransientException(msg);
|
||||
}
|
||||
rrs.setCacheAble(true);
|
||||
rrs.setSqlRouteCacheAble(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
296
src/main/java/com/actiontech/dble/route/util/ConditionUtil.java
Normal file
296
src/main/java/com/actiontech/dble/route/util/ConditionUtil.java
Normal file
@@ -0,0 +1,296 @@
|
||||
/*
|
||||
* Copyright (C) 2016-2020 ActionTech.
|
||||
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
|
||||
*/
|
||||
|
||||
package com.actiontech.dble.route.util;
|
||||
|
||||
import com.actiontech.dble.DbleServer;
|
||||
import com.actiontech.dble.config.model.TableConfig;
|
||||
import com.actiontech.dble.route.parser.druid.RouteCalculateUnit;
|
||||
import com.actiontech.dble.route.parser.druid.WhereUnit;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.server.util.SchemaUtil;
|
||||
import com.actiontech.dble.sqlengine.mpp.IsValue;
|
||||
import com.actiontech.dble.sqlengine.mpp.RangeValue;
|
||||
import com.actiontech.dble.util.StringUtil;
|
||||
import com.alibaba.druid.stat.TableStat;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
public final class ConditionUtil {
|
||||
private ConditionUtil() {
|
||||
}
|
||||
|
||||
private static void pruningConditions(List<WhereUnit> whereUnits, Map<String, String> tableAliasMap, String defaultSchema) {
|
||||
Iterator<WhereUnit> whereUnitIterator = whereUnits.listIterator();
|
||||
while (whereUnitIterator.hasNext()) {
|
||||
WhereUnit whereUnit = whereUnitIterator.next();
|
||||
final int subWhereSize = whereUnit.getSubWhereUnit().size();
|
||||
pruningConditions(whereUnit.getSubWhereUnit(), tableAliasMap, defaultSchema);
|
||||
final int subWhereSizeAfter = whereUnit.getSubWhereUnit().size();
|
||||
boolean orContainsEmpty = false;
|
||||
final int orSize = whereUnit.getOrConditionList().size();
|
||||
for (List<TableStat.Condition> conditions : whereUnit.getOrConditionList()) {
|
||||
pruningAndConditions(tableAliasMap, defaultSchema, conditions.listIterator());
|
||||
if (conditions.size() == 0) {
|
||||
orContainsEmpty = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (orContainsEmpty) {
|
||||
whereUnit.getOrConditionList().clear();
|
||||
}
|
||||
final int orSizeAfter = whereUnit.getOrConditionList().size();
|
||||
List<TableStat.Condition> outConditions = whereUnit.getOutAndConditions(); //outConditions item operator with AND
|
||||
ListIterator<TableStat.Condition> iteratorOutConditions = outConditions.listIterator();
|
||||
pruningAndConditions(tableAliasMap, defaultSchema, iteratorOutConditions);
|
||||
if (outConditions.size() == 0 && (subWhereSize != 0 && subWhereSizeAfter == 0) || (orSize != 0 && orSizeAfter == 0) || (subWhereSize == 0 && orSize == 0)) {
|
||||
whereUnitIterator.remove();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static void pruningAndConditions(Map<String, String> tableAliasMap, String defaultSchema, ListIterator<TableStat.Condition> iteratorConditions) {
|
||||
while (iteratorConditions.hasNext()) {
|
||||
TableStat.Condition condition = iteratorConditions.next();
|
||||
List<Object> values = condition.getValues();
|
||||
if (values.size() == 0 || !checkConditionValues(values)) {
|
||||
iteratorConditions.remove(); //AND CONDITION can be pruned
|
||||
} else {
|
||||
TableStat.Condition newCondition = getUsefulCondition(condition, tableAliasMap, defaultSchema);
|
||||
if (newCondition == null) {
|
||||
iteratorConditions.remove(); //AND CONDITION can be pruned
|
||||
} else {
|
||||
iteratorConditions.set(newCondition); //replace table name and column name
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static TableStat.Condition getUsefulCondition(TableStat.Condition condition, Map<String, String> tableAliasMap, String defaultSchema) {
|
||||
String tableFullName = condition.getColumn().getTable();
|
||||
if (DbleServer.getInstance().getSystemVariables().isLowerCaseTableNames()) {
|
||||
tableFullName = tableFullName.toLowerCase();
|
||||
}
|
||||
if (tableAliasMap != null && tableAliasMap.get(tableFullName) == null) {
|
||||
//ignore subQuery's alias
|
||||
return null;
|
||||
}
|
||||
|
||||
Pair<String, String> table = getTableInfo(tableAliasMap, tableFullName, defaultSchema);
|
||||
|
||||
String schemaName = table.getKey();
|
||||
String tableName = table.getValue();
|
||||
tableFullName = schemaName + "." + tableName;
|
||||
if (SchemaUtil.MYSQL_SYS_SCHEMA.contains(schemaName.toUpperCase())) {
|
||||
return null;
|
||||
}
|
||||
TableConfig tableConfig = DbleServer.getInstance().getConfig().getSchemas().get(schemaName).getTables().get(tableName);
|
||||
if (tableConfig == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
String operator = condition.getOperator();
|
||||
//execute only between ,in and = is
|
||||
if (!operator.equalsIgnoreCase("between") && !operator.equals("=") && !operator.equalsIgnoreCase("in") && !operator.equalsIgnoreCase("IS")) {
|
||||
return null;
|
||||
}
|
||||
String partitionCol = tableConfig.getPartitionColumn();
|
||||
|
||||
String columnName = StringUtil.removeBackQuote(condition.getColumn().getName().toUpperCase());
|
||||
if (columnName.equals(partitionCol)) {
|
||||
return genNewCondition(tableFullName, columnName, operator, condition.getValues());
|
||||
}
|
||||
|
||||
String joinKey = tableConfig.getJoinKey();
|
||||
if (joinKey != null && columnName.equals(joinKey)) {
|
||||
return genNewCondition(tableFullName, columnName, operator, condition.getValues());
|
||||
}
|
||||
String catchKey = tableConfig.getPrimaryKey();
|
||||
if (catchKey != null && columnName.equals(catchKey)) {
|
||||
return genNewCondition(tableFullName, columnName, operator, condition.getValues());
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private static Pair<String, String> getTableInfo(Map<String, String> tableAliasMap, String tableFullName, String defaultSchema) {
|
||||
if (tableAliasMap != null && tableAliasMap.get(tableFullName) != null &&
|
||||
!tableAliasMap.get(tableFullName).equals(tableFullName)) {
|
||||
tableFullName = tableAliasMap.get(tableFullName);
|
||||
}
|
||||
String schemaName;
|
||||
String tableName;
|
||||
int pos = tableFullName.indexOf(".");
|
||||
if (pos > 0) {
|
||||
tableName = tableFullName.substring(pos + 1);
|
||||
schemaName = tableFullName.substring(0, pos);
|
||||
} else {
|
||||
schemaName = defaultSchema;
|
||||
tableName = tableFullName;
|
||||
}
|
||||
return new Pair<>(schemaName, tableName);
|
||||
}
|
||||
|
||||
private static TableStat.Condition genNewCondition(String tableName, String columnName, String operator, List<Object> values) {
|
||||
TableStat.Column column = new TableStat.Column(tableName, columnName);
|
||||
TableStat.Condition condition = new TableStat.Condition(column, operator);
|
||||
for (Object value : values) {
|
||||
condition.addValue(value);
|
||||
}
|
||||
return condition;
|
||||
}
|
||||
|
||||
private static List<List<TableStat.Condition>> mergedConditions(List<WhereUnit> storedWhereUnits) {
|
||||
if (storedWhereUnits.size() == 0) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
for (WhereUnit whereUnit : storedWhereUnits) {
|
||||
mergeSubConditionWithOuterCondition(whereUnit);
|
||||
}
|
||||
|
||||
return getMergedConditionList(storedWhereUnits);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* mergeSubConditionWithOuterCondition
|
||||
* Only subWhereUnit will be deal
|
||||
*/
|
||||
private static void mergeSubConditionWithOuterCondition(WhereUnit whereUnit) {
|
||||
if (whereUnit.getSubWhereUnit().size() > 0) {
|
||||
for (WhereUnit sub : whereUnit.getSubWhereUnit()) {
|
||||
mergeSubConditionWithOuterCondition(sub);
|
||||
}
|
||||
List<List<TableStat.Condition>> mergedConditionList = getMergedConditionList(whereUnit.getSubWhereUnit());
|
||||
if (whereUnit.getOutAndConditions().size() > 0) {
|
||||
for (List<TableStat.Condition> mergedCondition : mergedConditionList) {
|
||||
mergedCondition.addAll(whereUnit.getOutAndConditions());
|
||||
}
|
||||
}
|
||||
whereUnit.getOrConditionList().addAll(mergedConditionList);
|
||||
} else if (whereUnit.getOutAndConditions().size() > 0) {
|
||||
whereUnit.getOrConditionList().add(whereUnit.getOutAndConditions());
|
||||
}
|
||||
}
|
||||
|
||||
private static List<List<TableStat.Condition>> getMergedConditionList(List<WhereUnit> whereUnitList) {
|
||||
List<List<TableStat.Condition>> mergedConditionList = new ArrayList<>();
|
||||
if (whereUnitList.size() == 0) {
|
||||
return mergedConditionList;
|
||||
}
|
||||
mergedConditionList.addAll(whereUnitList.get(0).getOrConditionList());
|
||||
|
||||
for (int i = 1; i < whereUnitList.size(); i++) {
|
||||
mergedConditionList = merge(mergedConditionList, whereUnitList.get(i).getOrConditionList());
|
||||
}
|
||||
return mergedConditionList;
|
||||
}
|
||||
|
||||
|
||||
private static List<List<TableStat.Condition>> merge(List<List<TableStat.Condition>> list1, List<List<TableStat.Condition>> list2) {
|
||||
if (list1.size() == 0) {
|
||||
return list2;
|
||||
} else if (list2.size() == 0) {
|
||||
return list1;
|
||||
}
|
||||
|
||||
List<List<TableStat.Condition>> retList = new ArrayList<>();
|
||||
for (List<TableStat.Condition> aList1 : list1) {
|
||||
for (List<TableStat.Condition> aList2 : list2) {
|
||||
List<TableStat.Condition> tmp = new ArrayList<>();
|
||||
tmp.addAll(aList1);
|
||||
tmp.addAll(aList2);
|
||||
retList.add(tmp);
|
||||
}
|
||||
}
|
||||
return retList;
|
||||
}
|
||||
|
||||
public static void extendConditionsFromRelations(List<TableStat.Condition> conds, Set<TableStat.Relationship> relations) {
|
||||
List<TableStat.Condition> newConds = new ArrayList<>();
|
||||
Iterator<TableStat.Condition> iterator = conds.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
TableStat.Condition condition = iterator.next();
|
||||
if (condition.getValues().size() == 0) {
|
||||
iterator.remove();
|
||||
continue;
|
||||
}
|
||||
if (!condition.getOperator().equals("=") && !condition.getOperator().equals("<=>")) {
|
||||
continue;
|
||||
}
|
||||
TableStat.Column column = condition.getColumn();
|
||||
for (TableStat.Relationship relation : relations) {
|
||||
if (!condition.getOperator().equalsIgnoreCase(relation.getOperator())) {
|
||||
continue;
|
||||
}
|
||||
if (column.equals(relation.getLeft())) {
|
||||
TableStat.Condition cond = new TableStat.Condition(relation.getRight(), condition.getOperator());
|
||||
cond.getValues().addAll(condition.getValues());
|
||||
newConds.add(cond);
|
||||
} else if (column.equals(relation.getRight())) {
|
||||
TableStat.Condition cond = new TableStat.Condition(relation.getLeft(), condition.getOperator());
|
||||
cond.getValues().addAll(condition.getValues());
|
||||
newConds.add(cond);
|
||||
}
|
||||
}
|
||||
}
|
||||
conds.addAll(newConds);
|
||||
}
|
||||
|
||||
private static List<RouteCalculateUnit> transformConditionToRouteUnits(List<List<TableStat.Condition>> conditionList) {
|
||||
List<RouteCalculateUnit> retList = new ArrayList<>();
|
||||
//find partition column in condition
|
||||
for (List<TableStat.Condition> aConditionList : conditionList) {
|
||||
RouteCalculateUnit routeCalculateUnit = new RouteCalculateUnit();
|
||||
for (TableStat.Condition condition : aConditionList) {
|
||||
List<Object> values = condition.getValues();
|
||||
String columnName = condition.getColumn().getName();
|
||||
String tableFullName = condition.getColumn().getTable();
|
||||
String operator = condition.getOperator();
|
||||
String[] tableInfo = tableFullName.split("\\.");
|
||||
Pair<String, String> table = new Pair<>(tableInfo[0], tableInfo[1]);
|
||||
//execute only between ,in and =
|
||||
if (operator.equalsIgnoreCase("between")) {
|
||||
RangeValue rv = new RangeValue(values.get(0), values.get(1), RangeValue.EE);
|
||||
routeCalculateUnit.addShardingExpr(table, columnName, rv);
|
||||
} else if (operator.equals("=")) {
|
||||
routeCalculateUnit.addShardingExpr(table, columnName, values.get(0));
|
||||
} else if (operator.equalsIgnoreCase("in")) {
|
||||
routeCalculateUnit.addShardingExpr(table, columnName, values.toArray());
|
||||
} else if (operator.equalsIgnoreCase("IS")) {
|
||||
IsValue isValue = new IsValue(values.toArray());
|
||||
routeCalculateUnit.addShardingExpr(table, columnName, isValue);
|
||||
}
|
||||
}
|
||||
retList.add(routeCalculateUnit);
|
||||
}
|
||||
return retList;
|
||||
}
|
||||
|
||||
private static boolean checkConditionValues(List<Object> values) {
|
||||
for (Object value : values) {
|
||||
if (value != null && !value.toString().equals("")) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
public static List<RouteCalculateUnit> buildRouteCalculateUnits(List<WhereUnit> whereUnits, Map<String, String> tableAliasMap, String defaultSchema) {
|
||||
ConditionUtil.pruningConditions(whereUnits, tableAliasMap, defaultSchema);
|
||||
if (whereUnits.size() == 0) {
|
||||
WhereUnit whereUnit = new WhereUnit();
|
||||
whereUnit.setFinishedParse(true);
|
||||
List<List<TableStat.Condition>> retList = new ArrayList<>();
|
||||
retList.add(new ArrayList<>());
|
||||
whereUnit.setOrConditionList(retList);
|
||||
whereUnits.add(whereUnit);
|
||||
}
|
||||
List<List<TableStat.Condition>> conditions = ConditionUtil.mergedConditions(whereUnits);
|
||||
return ConditionUtil.transformConditionToRouteUnits(conditions);
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -208,7 +208,7 @@ public final class ExplainHandler {
|
||||
return null;
|
||||
} else {
|
||||
StringBuilder s = new StringBuilder();
|
||||
LOGGER.info(s.append(c).append(stmt).toString() + " error:" + e);
|
||||
LOGGER.warn(s.append(c).append(stmt).toString() + " error:", e);
|
||||
String msg = e.getMessage();
|
||||
c.writeErrMessage(ErrorCode.ER_PARSE_ERROR, msg == null ? e.getClass().getSimpleName() : msg);
|
||||
return null;
|
||||
|
||||
@@ -37,8 +37,11 @@ public final class SelectHandler {
|
||||
case ServerParseSelect.SESSION_INCREMENT:
|
||||
SessionIncrement.response(c);
|
||||
break;
|
||||
case ServerParseSelect.SESSION_ISOLATION:
|
||||
SessionIsolation.response(c);
|
||||
case ServerParseSelect.SESSION_TX_ISOLATION:
|
||||
SessionIsolation.response(c, stmt.substring(offset).trim());
|
||||
break;
|
||||
case ServerParseSelect.SESSION_TRANSACTION_ISOLATION:
|
||||
SessionIsolation.response(c, stmt.substring(offset).trim());
|
||||
break;
|
||||
case ServerParseSelect.LAST_INSERT_ID:
|
||||
// offset = ParseUtil.move(stmt, 0, "select".length());
|
||||
@@ -90,7 +93,10 @@ public final class SelectHandler {
|
||||
SelectVariables.execute(c, stmt);
|
||||
break;
|
||||
case ServerParseSelect.SESSION_TX_READ_ONLY:
|
||||
SelectTxReadOnly.response(c);
|
||||
SelectTxReadOnly.response(c, stmt.substring(offset).trim());
|
||||
break;
|
||||
case ServerParseSelect.SESSION_TRANSACTION_READ_ONLY:
|
||||
SelectTxReadOnly.response(c, stmt.substring(offset).trim());
|
||||
break;
|
||||
case ServerParseSelect.TRACE:
|
||||
SelectTrace.response(c);
|
||||
|
||||
@@ -19,8 +19,8 @@ import com.actiontech.dble.net.mysql.OkPacket;
|
||||
import com.actiontech.dble.net.mysql.RequestFilePacket;
|
||||
import com.actiontech.dble.route.RouteResultset;
|
||||
import com.actiontech.dble.route.RouteResultsetNode;
|
||||
import com.actiontech.dble.route.parser.druid.DruidShardingParseInfo;
|
||||
import com.actiontech.dble.route.parser.druid.RouteCalculateUnit;
|
||||
import com.actiontech.dble.route.parser.util.Pair;
|
||||
import com.actiontech.dble.route.util.RouterUtil;
|
||||
import com.actiontech.dble.server.ServerConnection;
|
||||
import com.actiontech.dble.server.parser.ServerParse;
|
||||
@@ -37,14 +37,15 @@ import com.google.common.collect.Lists;
|
||||
import com.google.common.io.Files;
|
||||
import com.univocity.parsers.csv.CsvParser;
|
||||
import com.univocity.parsers.csv.CsvParserSettings;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.*;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.Charset;
|
||||
import java.sql.SQLException;
|
||||
import java.util.*;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
@@ -53,7 +54,6 @@ import java.util.regex.Pattern;
|
||||
* CHARACTER SET 'gbk' in load data sql the charset need ', otherwise the druid will error
|
||||
*/
|
||||
public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(ServerLoadDataInfileHandler.class);
|
||||
//innodb limit of columns per table, https://dev.mysql.com/doc/refman/8.0/en/column-count-limit.html
|
||||
private static final int DEFAULT_MAX_COLUMNS = 1017;
|
||||
private ServerConnection serverConnection;
|
||||
@@ -299,7 +299,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
}
|
||||
}
|
||||
|
||||
private RouteResultset tryDirectRoute(String strSql, String[] lineList) {
|
||||
private RouteResultset tryDirectRoute(String strSql, String[] lineList) throws SQLException {
|
||||
RouteResultset rrs = new RouteResultset(strSql, ServerParse.INSERT);
|
||||
rrs.setLoadData(true);
|
||||
if (tableConfig != null && tableConfig.isGlobalTable()) {
|
||||
@@ -314,56 +314,27 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
rrs.setNodes(rrsNodes);
|
||||
return rrs;
|
||||
} else {
|
||||
DruidShardingParseInfo ctx = new DruidShardingParseInfo();
|
||||
ctx.addTable(tableName);
|
||||
Pair<String, String> table = new Pair<>(schema.getName(), tableName);
|
||||
|
||||
if (partitionColumnIndex != -1) {
|
||||
String value;
|
||||
if (lineList.length < partitionColumnIndex + 1) {
|
||||
throw new RuntimeException("Partition column is empty in line '" + StringUtil.join(lineList, loadData.getLineTerminatedBy()) + "'");
|
||||
} else {
|
||||
value = lineList[partitionColumnIndex];
|
||||
}
|
||||
RouteCalculateUnit routeCalculateUnit = new RouteCalculateUnit();
|
||||
routeCalculateUnit.addShardingExpr(tableName, getPartitionColumn(),
|
||||
parseFieldString(value, loadData.getEnclose(), loadData.getEscape()));
|
||||
ctx.addRouteCalculateUnit(routeCalculateUnit);
|
||||
}
|
||||
|
||||
try {
|
||||
SortedSet<RouteResultsetNode> nodeSet = new TreeSet<>();
|
||||
if (ctx.getRouteCalculateUnits().size() > 0) {
|
||||
for (RouteCalculateUnit unit : ctx.getRouteCalculateUnits()) {
|
||||
RouteResultset rrsTmp = RouterUtil.tryRouteForTables(schema, ctx, unit, rrs, false, tableId2DataNodeCache, null);
|
||||
if (rrsTmp != null) {
|
||||
Collections.addAll(nodeSet, rrsTmp.getNodes());
|
||||
}
|
||||
}
|
||||
} else {
|
||||
RouteResultset rrsTmp = RouterUtil.tryRouteForTables(schema, ctx, null, rrs, false, tableId2DataNodeCache, null);
|
||||
if (rrsTmp != null) {
|
||||
Collections.addAll(nodeSet, rrsTmp.getNodes());
|
||||
}
|
||||
routeCalculateUnit.addShardingExpr(table, getPartitionColumn(), parseFieldString(lineList[partitionColumnIndex], loadData.getEnclose(), loadData.getEscape()));
|
||||
return RouterUtil.tryRouteForOneTable(schema, routeCalculateUnit, tableName, rrs, false, tableId2DataNodeCache, null);
|
||||
} else {
|
||||
String noShardingNode = RouterUtil.isNoSharding(schema, tableName);
|
||||
if (noShardingNode != null) {
|
||||
return RouterUtil.routeToSingleNode(rrs, noShardingNode);
|
||||
}
|
||||
|
||||
RouteResultsetNode[] nodes = new RouteResultsetNode[nodeSet.size()];
|
||||
int i = 0;
|
||||
for (RouteResultsetNode aNodeSet : nodeSet) {
|
||||
nodes[i] = aNodeSet;
|
||||
i++;
|
||||
}
|
||||
|
||||
rrs.setNodes(nodes);
|
||||
return rrs;
|
||||
} catch (SQLException e) {
|
||||
throw new RuntimeException(e);
|
||||
return RouterUtil.tryRouteForOneTable(schema, new RouteCalculateUnit(), tableName, rrs, false, tableId2DataNodeCache, null);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
private void parseOneLine(String[] line, boolean toFile) throws Exception {
|
||||
private void parseOneLine(String[] line) throws Exception {
|
||||
if (loadData.getEnclose() != null && loadData.getEnclose().charAt(0) > 0x0020) {
|
||||
for (int i = 0; i < line.length; i++) {
|
||||
line[i] = line[i].trim();
|
||||
@@ -396,7 +367,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
data.getData().add(jLine);
|
||||
}
|
||||
|
||||
if (toFile && data.getData().size() > systemConfig.getMaxRowSizeToFile()) {
|
||||
if (data.getData().size() > systemConfig.getMaxRowSizeToFile()) {
|
||||
//avoid OOM
|
||||
saveDataToFile(data, name);
|
||||
}
|
||||
@@ -473,7 +444,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
rrs.setLoadData(true);
|
||||
rrs.setStatement(srcStatement);
|
||||
rrs.setFinishedRoute(true);
|
||||
rrs.setGlobalTable(tableConfig == null ? false : this.tableConfig.isGlobalTable());
|
||||
rrs.setGlobalTable(tableConfig != null && this.tableConfig.isGlobalTable());
|
||||
int size = routeMap.size();
|
||||
RouteResultsetNode[] routeResultsetNodes = new RouteResultsetNode[size];
|
||||
int index = 0;
|
||||
@@ -502,7 +473,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
private String parseFieldString(String value, String enclose, String escape) {
|
||||
//avoid null point execption
|
||||
if (value == null) {
|
||||
return value;
|
||||
return null;
|
||||
}
|
||||
|
||||
//if the value is cover by enclose char and enclose char is not null, clear the enclose char.
|
||||
@@ -597,7 +568,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
CsvParser parser = new CsvParser(settings);
|
||||
try {
|
||||
parser.beginParsing(new StringReader(content));
|
||||
String[] row = null;
|
||||
String[] row;
|
||||
|
||||
int ignoreNumber = 0;
|
||||
if (statement.getIgnoreLinesNumber() != null && !"".equals(statement.getIgnoreLinesNumber().toString())) {
|
||||
@@ -609,7 +580,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
continue;
|
||||
}
|
||||
try {
|
||||
parseOneLine(row, true);
|
||||
parseOneLine(row);
|
||||
} catch (Exception e) {
|
||||
clear();
|
||||
serverConnection.writeErrMessage(++packId, ErrorCode.ER_WRONG_VALUE_COUNT_ON_ROW, "row data can't not calculate a sharding value," + e.getMessage());
|
||||
@@ -656,7 +627,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
fileInputStream = new FileInputStream(file);
|
||||
reader = new InputStreamReader(fileInputStream, encode);
|
||||
parser.beginParsing(reader);
|
||||
String[] row = null;
|
||||
String[] row;
|
||||
|
||||
int ignoreNumber = 0;
|
||||
if (statement.getIgnoreLinesNumber() != null && !"".equals(statement.getIgnoreLinesNumber().toString())) {
|
||||
@@ -669,7 +640,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
continue;
|
||||
}
|
||||
try {
|
||||
parseOneLine(row, true);
|
||||
parseOneLine(row);
|
||||
} catch (Exception e) {
|
||||
clear();
|
||||
serverConnection.writeErrMessage(++packetID, ErrorCode.ER_WRONG_VALUE_COUNT_ON_ROW, "row data can't not calculate a sharding value," + e.getMessage());
|
||||
@@ -715,8 +686,6 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
/**
|
||||
* check if the sql is contain the partition. If the sql contain the partition word then stopped.
|
||||
*
|
||||
* @param strSql
|
||||
* @throws Exception
|
||||
*/
|
||||
private boolean checkPartition(String strSql) {
|
||||
Pattern p = Pattern.compile("PARTITION\\s{0,}([\\s\\S]*)", Pattern.CASE_INSENSITIVE);
|
||||
@@ -728,8 +697,6 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
/**
|
||||
* use a Regular Expression to replace the "IGNORE 1234 LINES" to the " "
|
||||
*
|
||||
* @param strSql
|
||||
* @return
|
||||
*/
|
||||
private String ignoreLinesDelete(String strSql) {
|
||||
Pattern p = Pattern.compile("IGNORE\\s{0,}\\d{0,}\\s{0,}LINES", Pattern.CASE_INSENSITIVE);
|
||||
@@ -799,8 +766,6 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
/**
|
||||
* deleteFile and its children
|
||||
*
|
||||
* @param dirPath
|
||||
* @throws Exception
|
||||
*/
|
||||
private static void deleteFile(String dirPath) {
|
||||
File fileDirToDel = new File(dirPath);
|
||||
@@ -815,7 +780,7 @@ public final class ServerLoadDataInfileHandler implements LoadDataInfileHandler
|
||||
if (fileList != null) {
|
||||
for (File file : fileList) {
|
||||
if (file.isFile() && file.exists()) {
|
||||
boolean delete = file.delete();
|
||||
file.delete();
|
||||
} else if (file.isDirectory()) {
|
||||
deleteFile(file.getAbsolutePath());
|
||||
file.delete();
|
||||
|
||||
@@ -8,6 +8,7 @@ package com.actiontech.dble.server.handler;
|
||||
import com.actiontech.dble.DbleServer;
|
||||
import com.actiontech.dble.backend.datasource.PhysicalDBPool;
|
||||
import com.actiontech.dble.backend.mysql.CharsetUtil;
|
||||
import com.actiontech.dble.backend.mysql.VersionUtil;
|
||||
import com.actiontech.dble.config.ErrorCode;
|
||||
import com.actiontech.dble.config.Isolations;
|
||||
import com.actiontech.dble.log.transaction.TxnLogHelper;
|
||||
@@ -292,7 +293,7 @@ public final class SetHandler {
|
||||
String value = parseStringValue(valueExpr);
|
||||
Integer txIsolation = getIsolationLevel(value);
|
||||
if (txIsolation == null) {
|
||||
c.writeErrMessage(ErrorCode.ERR_NOT_SUPPORTED, "Variable 'tx_isolation' can't be set to the value of '" + value + "'");
|
||||
c.writeErrMessage(ErrorCode.ERR_NOT_SUPPORTED, "Variable 'tx_isolation|transaction_isolation' can't be set to the value of '" + value + "'");
|
||||
return false;
|
||||
}
|
||||
contextTask.add(new Pair<>(KeyType.TX_ISOLATION, new Pair<String, String>(String.valueOf(txIsolation), null)));
|
||||
@@ -447,7 +448,7 @@ public final class SetHandler {
|
||||
String value = parseStringValue(valueExpr);
|
||||
Integer txIsolation = getIsolationLevel(value);
|
||||
if (txIsolation == null) {
|
||||
c.writeErrMessage(ErrorCode.ERR_NOT_SUPPORTED, "Variable 'tx_isolation' can't be set to the value of '" + value + "'");
|
||||
c.writeErrMessage(ErrorCode.ERR_NOT_SUPPORTED, "Variable 'tx_isolation|transaction_isolation' can't be set to the value of '" + value + "'");
|
||||
return false;
|
||||
}
|
||||
c.setTxIsolation(txIsolation);
|
||||
@@ -668,11 +669,11 @@ public final class SetHandler {
|
||||
return KeyType.CHARACTER_SET_RESULTS;
|
||||
case "character_set_connection":
|
||||
return KeyType.CHARACTER_SET_CONNECTION;
|
||||
case "transaction_isolation":
|
||||
case "tx_isolation":
|
||||
case VersionUtil.TRANSACTION_ISOLATION:
|
||||
case VersionUtil.TX_ISOLATION:
|
||||
return KeyType.TX_ISOLATION;
|
||||
case "transaction_read_only":
|
||||
case "tx_read_only":
|
||||
case VersionUtil.TRANSACTION_READ_ONLY:
|
||||
case VersionUtil.TX_READ_ONLY:
|
||||
return KeyType.TX_READ_ONLY;
|
||||
case "names":
|
||||
return KeyType.NAMES;
|
||||
|
||||
@@ -5,6 +5,7 @@
|
||||
*/
|
||||
package com.actiontech.dble.server.parser;
|
||||
|
||||
import com.actiontech.dble.backend.mysql.VersionUtil;
|
||||
import com.actiontech.dble.route.parser.util.CharTypes;
|
||||
import com.actiontech.dble.route.parser.util.ParseUtil;
|
||||
|
||||
@@ -23,10 +24,13 @@ public final class ServerParseSelect {
|
||||
public static final int IDENTITY = 5;
|
||||
public static final int VERSION = 6;
|
||||
public static final int SESSION_INCREMENT = 7;
|
||||
public static final int SESSION_ISOLATION = 8;
|
||||
public static final int SESSION_TX_ISOLATION = 8;
|
||||
public static final int SELECT_VAR_ALL = 9;
|
||||
public static final int SESSION_TX_READ_ONLY = 10;
|
||||
public static final int TRACE = 11;
|
||||
public static final int CURRENT_USER = 12;
|
||||
public static final int SESSION_TRANSACTION_ISOLATION = 13;
|
||||
public static final int SESSION_TRANSACTION_READ_ONLY = 14;
|
||||
|
||||
private static final char[] TRACE_STR = "TRACE".toCharArray();
|
||||
private static final char[] VERSION_COMMENT_STR = "VERSION_COMMENT".toCharArray();
|
||||
@@ -93,15 +97,23 @@ public final class ServerParseSelect {
|
||||
*/
|
||||
private static int sessionVarCheck(String stmt, int offset) {
|
||||
String s = stmt.substring(offset).toLowerCase();
|
||||
if (s.startsWith("session.auto_increment_increment")) {
|
||||
if (!s.startsWith("session.")) {
|
||||
return OTHER;
|
||||
}
|
||||
s = s.substring(8);
|
||||
if (s.startsWith("auto_increment_increment")) {
|
||||
if (s.contains("@@")) {
|
||||
return SELECT_VAR_ALL;
|
||||
}
|
||||
return SESSION_INCREMENT;
|
||||
} else if (s.startsWith("session.tx_isolation")) {
|
||||
return SESSION_ISOLATION;
|
||||
} else if (s.startsWith("session.tx_read_only")) {
|
||||
} else if (s.startsWith(VersionUtil.TX_ISOLATION)) {
|
||||
return SESSION_TX_ISOLATION;
|
||||
} else if (s.startsWith(VersionUtil.TRANSACTION_ISOLATION)) {
|
||||
return SESSION_TRANSACTION_ISOLATION;
|
||||
} else if (s.startsWith(VersionUtil.TX_READ_ONLY)) {
|
||||
return SESSION_TX_READ_ONLY;
|
||||
} else if (s.startsWith(VersionUtil.TRANSACTION_READ_ONLY)) {
|
||||
return SESSION_TRANSACTION_READ_ONLY;
|
||||
} else {
|
||||
return OTHER;
|
||||
}
|
||||
|
||||
@@ -28,19 +28,12 @@ public final class SelectTxReadOnly {
|
||||
private static final FieldPacket[] FIELDS = new FieldPacket[FIELD_COUNT];
|
||||
private static final EOFPacket EOF = new EOFPacket();
|
||||
|
||||
static {
|
||||
int i = 0;
|
||||
byte packetId = 0;
|
||||
|
||||
|
||||
}
|
||||
|
||||
public static void response(ServerConnection c) {
|
||||
public static void response(ServerConnection c, String column) {
|
||||
|
||||
byte packetId = setCurrentPacket(c);
|
||||
|
||||
HEADER.setPacketId(++packetId);
|
||||
FIELDS[0] = PacketUtil.getField("@@session.tx_read_only", Fields.FIELD_TYPE_LONG);
|
||||
FIELDS[0] = PacketUtil.getField(column, Fields.FIELD_TYPE_LONG);
|
||||
FIELDS[0].setPacketId(++packetId);
|
||||
EOF.setPacketId(++packetId);
|
||||
|
||||
|
||||
@@ -6,6 +6,7 @@
|
||||
package com.actiontech.dble.server.response;
|
||||
|
||||
import com.actiontech.dble.backend.mysql.PacketUtil;
|
||||
import com.actiontech.dble.backend.mysql.VersionUtil;
|
||||
import com.actiontech.dble.config.Fields;
|
||||
import com.actiontech.dble.net.mysql.EOFPacket;
|
||||
import com.actiontech.dble.net.mysql.FieldPacket;
|
||||
@@ -127,7 +128,8 @@ public final class SelectVariables {
|
||||
VARIABLES.put("@@sql_mode", "STRICT_TRANS_TABLES");
|
||||
VARIABLES.put("@@system_time_zone", "CST");
|
||||
VARIABLES.put("@@time_zone", "SYSTEM");
|
||||
VARIABLES.put("@@tx_isolation", "REPEATABLE-READ");
|
||||
VARIABLES.put("@@" + VersionUtil.TRANSACTION_ISOLATION, "REPEATABLE-READ");
|
||||
VARIABLES.put("@@" + VersionUtil.TX_ISOLATION, "REPEATABLE-READ");
|
||||
VARIABLES.put("@@wait_timeout", "172800");
|
||||
VARIABLES.put("@@session.auto_increment_increment", "1");
|
||||
|
||||
@@ -147,7 +149,8 @@ public final class SelectVariables {
|
||||
VARIABLES.put("sql_mode", "STRICT_TRANS_TABLES");
|
||||
VARIABLES.put("system_time_zone", "CST");
|
||||
VARIABLES.put("time_zone", "SYSTEM");
|
||||
VARIABLES.put("tx_isolation", "REPEATABLE-READ");
|
||||
VARIABLES.put(VersionUtil.TRANSACTION_ISOLATION, "REPEATABLE-READ");
|
||||
VARIABLES.put(VersionUtil.TX_ISOLATION, "REPEATABLE-READ");
|
||||
VARIABLES.put("wait_timeout", "172800");
|
||||
VARIABLES.put("auto_increment_increment", "1");
|
||||
}
|
||||
|
||||
@@ -30,10 +30,10 @@ public final class SessionIsolation {
|
||||
private static final EOFPacket EOF = new EOFPacket();
|
||||
|
||||
|
||||
public static void response(ServerConnection c) {
|
||||
public static void response(ServerConnection c, String column) {
|
||||
byte packetId = setCurrentPacket(c);
|
||||
HEADER.setPacketId(++packetId);
|
||||
FIELDS[0] = PacketUtil.getField("@@session.tx_isolation", Fields.FIELD_TYPE_STRING);
|
||||
FIELDS[0] = PacketUtil.getField(column, Fields.FIELD_TYPE_STRING);
|
||||
FIELDS[0].setPacketId(++packetId);
|
||||
EOF.setPacketId(++packetId);
|
||||
|
||||
|
||||
@@ -5,6 +5,7 @@
|
||||
|
||||
package com.actiontech.dble.server.variables;
|
||||
|
||||
import com.actiontech.dble.backend.mysql.VersionUtil;
|
||||
import com.actiontech.dble.util.StringUtil;
|
||||
|
||||
import java.util.HashMap;
|
||||
@@ -207,8 +208,10 @@ public final class SystemVariables {
|
||||
sessionVariables.put("transaction_allow_batching", "0");
|
||||
sessionVariables.put("transaction_prealloc_size", "4096");
|
||||
sessionVariables.put("transaction_write_set_extraction", "0");
|
||||
sessionVariables.put("tx_isolation", "REPEATABLE-READ"); //transaction-isolation
|
||||
sessionVariables.put("tx_read_only", "0"); // OFF|0|false //transaction-read-only
|
||||
sessionVariables.put(VersionUtil.TX_ISOLATION, "REPEATABLE-READ"); //transaction-isolation
|
||||
sessionVariables.put(VersionUtil.TRANSACTION_ISOLATION, "REPEATABLE-READ"); //transaction-isolation
|
||||
sessionVariables.put(VersionUtil.TX_READ_ONLY, "0"); // OFF|0|false //transaction-read-only
|
||||
sessionVariables.put(VersionUtil.TRANSACTION_READ_ONLY, "0"); // OFF|0|false //transaction-read-only
|
||||
sessionVariables.put("unique_checks", "1"); // ON|1|TRUE
|
||||
sessionVariables.put("updatable_views_with_limit", "1"); // ON|1|TRUE
|
||||
sessionVariables.put("version_tokens_session", "NULL");
|
||||
|
||||
@@ -57,6 +57,10 @@ public class SqlFrequency implements Comparable<SqlFrequency> {
|
||||
return avgTime;
|
||||
}
|
||||
|
||||
public long getAllExecuteTime() {
|
||||
return allExecuteTime;
|
||||
}
|
||||
|
||||
public void setExecuteTime(long execTime) {
|
||||
if (execTime > this.maxTime) {
|
||||
this.maxTime = execTime;
|
||||
@@ -76,7 +80,9 @@ public class SqlFrequency implements Comparable<SqlFrequency> {
|
||||
|
||||
@Override
|
||||
public int compareTo(SqlFrequency o) {
|
||||
long para = o.count.get() - count.get();
|
||||
//long para = o.count.get() - count.get();
|
||||
// use lastTime for UserSqlHighStat.sqlFrequencySortedSet
|
||||
long para = 0L;
|
||||
long para2 = o.lastTime - lastTime;
|
||||
return para == 0L ? (int) (para2 == 0L ? o.allExecuteTime - allExecuteTime : para2) : (int) para;
|
||||
}
|
||||
|
||||
@@ -9,7 +9,6 @@ import com.alibaba.druid.sql.visitor.ParameterizedOutputVisitorUtils;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
public class UserSqlHighStat {
|
||||
|
||||
@@ -17,33 +16,23 @@ public class UserSqlHighStat {
|
||||
|
||||
private Map<String, SqlFrequency> sqlFrequencyMap = new ConcurrentHashMap<>();
|
||||
|
||||
private ReentrantLock lock = new ReentrantLock();
|
||||
|
||||
|
||||
private SqlParser sqlParser = new SqlParser();
|
||||
|
||||
public void addSql(String sql, long executeTime, long startTime, long endTime) {
|
||||
String newSql = this.sqlParser.mergeSql(sql);
|
||||
SqlFrequency frequency = this.sqlFrequencyMap.get(newSql);
|
||||
if (frequency == null) {
|
||||
if (lock.tryLock()) {
|
||||
try {
|
||||
frequency = new SqlFrequency();
|
||||
frequency.setSql(newSql);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
} else {
|
||||
while (frequency == null) {
|
||||
frequency = this.sqlFrequencyMap.get(newSql);
|
||||
}
|
||||
frequency = new SqlFrequency();
|
||||
frequency.setSql(newSql);
|
||||
SqlFrequency tmp = sqlFrequencyMap.putIfAbsent(newSql, frequency);
|
||||
if (tmp != null) {
|
||||
frequency = tmp;
|
||||
}
|
||||
}
|
||||
frequency.setLastTime(endTime);
|
||||
frequency.incCount();
|
||||
//TODO setExecuteTime has thread safe problem
|
||||
frequency.setExecuteTime(executeTime);
|
||||
this.sqlFrequencyMap.put(newSql, frequency);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@@ -6,8 +6,6 @@
|
||||
package com.actiontech.dble.statistic.stat;
|
||||
|
||||
import com.actiontech.dble.DbleServer;
|
||||
import com.actiontech.dble.server.parser.ServerParse;
|
||||
import com.actiontech.dble.statistic.SQLRecord;
|
||||
import com.actiontech.dble.statistic.SQLRecorder;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
@@ -19,7 +17,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||
*/
|
||||
public class UserStat {
|
||||
|
||||
private volatile int sqlSlowTime = DbleServer.getInstance().getConfig().getSystem().getSqlSlowTime();
|
||||
//private volatile int sqlSlowTime = DbleServer.getInstance().getConfig().getSystem().getSqlSlowTime();
|
||||
|
||||
private String user;
|
||||
|
||||
@@ -102,10 +100,10 @@ public class UserStat {
|
||||
}
|
||||
|
||||
|
||||
public void setSlowTime(int time) {
|
||||
this.sqlSlowTime = time;
|
||||
this.sqlRecorder.clear();
|
||||
}
|
||||
//public void setSlowTime(int time) {
|
||||
// this.sqlSlowTime = time;
|
||||
// this.sqlRecorder.clear();
|
||||
//}
|
||||
|
||||
public void clearSql() {
|
||||
this.sqlLastStat.reset();
|
||||
@@ -155,33 +153,33 @@ public class UserStat {
|
||||
|
||||
//slow sql
|
||||
long executeTime = endTime - startTime;
|
||||
if (executeTime >= sqlSlowTime) {
|
||||
SQLRecord record = new SQLRecord();
|
||||
record.setExecuteTime(executeTime);
|
||||
record.setStatement(sql);
|
||||
record.setStartTime(startTime);
|
||||
this.sqlRecorder.add(record);
|
||||
}
|
||||
//if (executeTime >= sqlSlowTime) {
|
||||
// SQLRecord record = new SQLRecord();
|
||||
// record.setExecuteTime(executeTime);
|
||||
// record.setStatement(sql);
|
||||
// record.setStartTime(startTime);
|
||||
// this.sqlRecorder.add(record);
|
||||
//}
|
||||
|
||||
//sqlRwStat
|
||||
this.sqlRwStat.setConcurrentMax(concurrentMax.get());
|
||||
this.sqlRwStat.add(sqlType, sql, executeTime, netInBytes, netOutBytes, startTime, endTime);
|
||||
//this.sqlRwStat.setConcurrentMax(concurrentMax.get());
|
||||
//this.sqlRwStat.add(sqlType, sql, executeTime, netInBytes, netOutBytes, startTime, endTime);
|
||||
|
||||
//sqlLastStatSQL
|
||||
this.sqlLastStat.add(sql, executeTime, startTime, endTime);
|
||||
//this.sqlLastStat.add(sql, executeTime, startTime, endTime);
|
||||
|
||||
//sqlHighStat
|
||||
this.sqlHighStat.addSql(sql, executeTime, startTime, endTime);
|
||||
|
||||
//sqlLargeStat large than 10000 rows
|
||||
if (sqlType == ServerParse.SELECT && sqlRows > 10000) {
|
||||
this.sqlLargeStat.add(sql, sqlRows, executeTime, startTime, endTime);
|
||||
}
|
||||
//if (sqlType == ServerParse.SELECT && sqlRows > 10000) {
|
||||
// this.sqlLargeStat.add(sql, sqlRows, executeTime, startTime, endTime);
|
||||
//}
|
||||
|
||||
//big size sql
|
||||
if (resultSetSize >= DbleServer.getInstance().getConfig().getSystem().getMaxResultSet()) {
|
||||
this.sqlResultSizeRecorder.addSql(sql, resultSetSize);
|
||||
}
|
||||
//if (resultSetSize >= DbleServer.getInstance().getConfig().getSystem().getMaxResultSet()) {
|
||||
// this.sqlResultSizeRecorder.addSql(sql, resultSetSize);
|
||||
//}
|
||||
|
||||
//after
|
||||
//-----------------------------------------------------
|
||||
|
||||
@@ -6,7 +6,7 @@
|
||||
package com.actiontech.dble.buffer;
|
||||
|
||||
import jdk.nashorn.internal.ir.annotations.Ignore;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import sun.nio.ch.DirectBuffer;
|
||||
|
||||
|
||||
@@ -6,7 +6,7 @@
|
||||
package com.actiontech.dble.cache;
|
||||
|
||||
import com.actiontech.dble.cache.impl.EnchachePooFactory;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class DefaultLayedCachePoolTest {
|
||||
|
||||
@@ -6,7 +6,7 @@
|
||||
package com.actiontech.dble.cache;
|
||||
|
||||
import com.actiontech.dble.cache.impl.EnchachePool;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import net.sf.ehcache.Cache;
|
||||
import net.sf.ehcache.CacheManager;
|
||||
import net.sf.ehcache.config.CacheConfiguration;
|
||||
|
||||
@@ -6,14 +6,14 @@
|
||||
package com.actiontech.dble.parser.druid;
|
||||
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.druid.WhereUnit;
|
||||
import com.alibaba.druid.sql.ast.SQLStatement;
|
||||
import com.alibaba.druid.sql.dialect.mysql.parser.MySqlStatementParser;
|
||||
import com.alibaba.druid.sql.parser.SQLStatementParser;
|
||||
import com.alibaba.druid.stat.TableStat.Condition;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
@@ -32,32 +32,28 @@ public class ServerSchemaStatVisitorTest {
|
||||
String sql = "select id from travelrecord "
|
||||
+ " where id = 1 and ( fee=3 or days=5 or (traveldate = '2015-05-04 00:00:07.375' "
|
||||
+ " and (user_id=2 or fee=days or fee = 0))) and id=2";
|
||||
List<List<Condition>> list = getConditionList(sql);
|
||||
Assert.assertEquals(list.size(), 5);
|
||||
Assert.assertEquals(list.get(0).size(), 2);
|
||||
Assert.assertEquals(list.get(1).size(), 2);
|
||||
Assert.assertEquals(list.get(2).size(), 3);
|
||||
Assert.assertEquals(list.get(3).size(), 2);
|
||||
Assert.assertEquals(list.get(4).size(), 3);
|
||||
List<WhereUnit> whereUnits = getAllWhereUnit(sql);
|
||||
WhereUnit whereUnit = whereUnits.get(0);
|
||||
List<Condition> list1 = whereUnit.getOutAndConditions();
|
||||
|
||||
Assert.assertEquals(list.get(0).get(0).toString(), "travelrecord.days = 5");
|
||||
Assert.assertEquals(list.get(0).get(1).toString(), "travelrecord.id = (1, 2)");
|
||||
Assert.assertEquals(list1.get(0).toString(), "travelrecord.id = (1, 2)");
|
||||
|
||||
Assert.assertEquals(list.get(1).get(0).toString(), "travelrecord.fee = 3");
|
||||
Assert.assertEquals(list.get(1).get(1).toString(), "travelrecord.id = (1, 2)");
|
||||
WhereUnit childWhereUnits = whereUnit.getSubWhereUnit().get(0);
|
||||
List<List<Condition>> childList = childWhereUnits.getOrConditionList();
|
||||
|
||||
Assert.assertEquals(list.get(2).get(0).toString(), "travelrecord.fee = 0");
|
||||
Assert.assertEquals(list.get(2).get(1).toString(), "travelrecord.traveldate = 2015-05-04 00:00:07.375");
|
||||
Assert.assertEquals(list.get(2).get(2).toString(), "travelrecord.id = (1, 2)");
|
||||
Assert.assertEquals(childList.get(0).get(0).toString(), "travelrecord.days = 5");
|
||||
Assert.assertEquals(childList.get(1).get(0).toString(), "travelrecord.fee = 3");
|
||||
|
||||
Assert.assertEquals(list.get(3).get(0).toString(), "travelrecord.traveldate = 2015-05-04 00:00:07.375");
|
||||
Assert.assertEquals(list.get(3).get(1).toString(), "travelrecord.id = (1, 2)");
|
||||
WhereUnit child2WhereUnits = childWhereUnits.getSubWhereUnit().get(0);
|
||||
List<Condition> child2 = child2WhereUnits.getOutAndConditions();
|
||||
Assert.assertEquals(child2.get(0).toString(), "travelrecord.traveldate = 2015-05-04 00:00:07.375");
|
||||
|
||||
Assert.assertEquals(list.get(4).get(0).toString(), "travelrecord.user_id = 2");
|
||||
Assert.assertEquals(list.get(4).get(1).toString(), "travelrecord.traveldate = 2015-05-04 00:00:07.375");
|
||||
Assert.assertEquals(list.get(4).get(2).toString(), "travelrecord.id = (1, 2)");
|
||||
WhereUnit child3WhereUnits = child2WhereUnits.getSubWhereUnit().get(0);
|
||||
List<List<Condition>> child3List = child3WhereUnits.getOrConditionList();
|
||||
Assert.assertEquals(child3List.get(0).get(0).toString(), "travelrecord.fee = 0");
|
||||
Assert.assertEquals(child3List.get(1).size(), 0);
|
||||
Assert.assertEquals(child3List.get(2).get(0).toString(), "travelrecord.user_id = 2");
|
||||
|
||||
System.out.println(list.size());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -67,22 +63,18 @@ public class ServerSchemaStatVisitorTest {
|
||||
public void test2() {
|
||||
String sql = "select id from travelrecord "
|
||||
+ " where id = 1 and ( fee=3 or days=5 or name = 'zhangsan')";
|
||||
List<List<Condition>> list = getConditionList(sql);
|
||||
List<WhereUnit> whereUnits = getAllWhereUnit(sql);
|
||||
WhereUnit whereUnit = whereUnits.get(0);
|
||||
List<Condition> list1 = whereUnit.getOutAndConditions();
|
||||
Assert.assertEquals(list1.get(0).toString(), "travelrecord.id = 1");
|
||||
|
||||
Assert.assertEquals(list.size(), 3);
|
||||
Assert.assertEquals(list.get(0).size(), 2);
|
||||
Assert.assertEquals(list.get(1).size(), 2);
|
||||
Assert.assertEquals(list.get(2).size(), 2);
|
||||
WhereUnit childWhereUnits = whereUnit.getSubWhereUnit().get(0);
|
||||
List<List<Condition>> childList = childWhereUnits.getOrConditionList();
|
||||
|
||||
Assert.assertEquals(childList.get(0).get(0).toString(), "travelrecord.name = zhangsan");
|
||||
Assert.assertEquals(childList.get(1).get(0).toString(), "travelrecord.days = 5");
|
||||
Assert.assertEquals(childList.get(2).get(0).toString(), "travelrecord.fee = 3");
|
||||
|
||||
Assert.assertEquals(list.get(0).get(0).toString(), "travelrecord.name = zhangsan");
|
||||
Assert.assertEquals(list.get(0).get(1).toString(), "travelrecord.id = 1");
|
||||
|
||||
Assert.assertEquals(list.get(1).get(0).toString(), "travelrecord.days = 5");
|
||||
Assert.assertEquals(list.get(1).get(1).toString(), "travelrecord.id = 1");
|
||||
|
||||
Assert.assertEquals(list.get(2).get(0).toString(), "travelrecord.fee = 3");
|
||||
Assert.assertEquals(list.get(2).get(1).toString(), "travelrecord.id = 1");
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -92,8 +84,8 @@ public class ServerSchemaStatVisitorTest {
|
||||
public void test3() {
|
||||
String sql = "select id from travelrecord "
|
||||
+ " where id = 1 and fee=3 or days=5 or name = 'zhangsan'";
|
||||
List<List<Condition>> list = getConditionList(sql);
|
||||
|
||||
List<WhereUnit> whereUnits = getAllWhereUnit(sql);
|
||||
List<List<Condition>> list = whereUnits.get(0).getOrConditionList();
|
||||
Assert.assertEquals(list.size(), 3);
|
||||
|
||||
Assert.assertEquals(list.get(0).size(), 1);
|
||||
@@ -108,18 +100,16 @@ public class ServerSchemaStatVisitorTest {
|
||||
Assert.assertEquals(list.get(2).get(1).toString(), "travelrecord.fee = 3");
|
||||
}
|
||||
|
||||
private List<List<Condition>> getConditionList(String sql) {
|
||||
SQLStatementParser parser = null;
|
||||
parser = new MySqlStatementParser(sql);
|
||||
private List<WhereUnit> getAllWhereUnit(String sql) {
|
||||
SQLStatementParser parser = new MySqlStatementParser(sql);
|
||||
|
||||
ServerSchemaStatVisitor visitor = null;
|
||||
SQLStatement statement = null;
|
||||
ServerSchemaStatVisitor visitor;
|
||||
//throw exception
|
||||
try {
|
||||
statement = parser.parseStatement();
|
||||
SQLStatement statement = parser.parseStatement();
|
||||
visitor = new ServerSchemaStatVisitor();
|
||||
statement.accept(visitor);
|
||||
return visitor.getConditionList();
|
||||
return visitor.getAllWhereUnit();
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
@@ -5,7 +5,7 @@
|
||||
|
||||
package com.actiontech.dble.performance;
|
||||
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
@@ -16,7 +16,7 @@ import com.actiontech.dble.server.parser.ServerParse;
|
||||
import com.alibaba.druid.sql.ast.statement.SQLCreateTableStatement;
|
||||
import com.alibaba.druid.sql.dialect.mysql.parser.MySqlStatementParser;
|
||||
import com.alibaba.druid.sql.parser.SQLStatementParser;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
@@ -366,7 +366,7 @@ public class DDLRouteTest {
|
||||
String sql = "desc offer";
|
||||
RouteResultset rrs = routeStrategy.route(schema, ServerParse.DESCRIBE, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -375,7 +375,7 @@ public class DDLRouteTest {
|
||||
|
||||
sql = " desc cndb.offer";
|
||||
rrs = routeStrategy.route(schema, ServerParse.DESCRIBE, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -384,7 +384,7 @@ public class DDLRouteTest {
|
||||
|
||||
sql = " desc cndb.offer col1";
|
||||
rrs = routeStrategy.route(schema, ServerParse.DESCRIBE, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -394,7 +394,7 @@ public class DDLRouteTest {
|
||||
sql = "SHOW FULL COLUMNS FROM offer IN db_name WHERE true";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -406,7 +406,7 @@ public class DDLRouteTest {
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
// Assert.assertEquals("offer_dn[0]", rrs.getNodes()[0].getName());
|
||||
@@ -418,13 +418,13 @@ public class DDLRouteTest {
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("SHOW FULL TABLES WHERE Table_type != 'VIEW'", rrs.getNodes()[0].getStatement());
|
||||
|
||||
sql = "SHOW INDEX IN offer FROM db_name";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
|
||||
@@ -1,127 +0,0 @@
|
||||
/*
|
||||
* Copyright (C) 2016-2019 ActionTech.
|
||||
* License: http://www.gnu.org/licenses/gpl.html GPL version 2 or higher.
|
||||
*/
|
||||
|
||||
package com.actiontech.dble.route;
|
||||
|
||||
import com.actiontech.dble.SimpleCachePool;
|
||||
import com.actiontech.dble.cache.LayerCachePool;
|
||||
import com.actiontech.dble.config.loader.SchemaLoader;
|
||||
import com.actiontech.dble.config.loader.xml.XMLSchemaLoader;
|
||||
import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.route.factory.RouteStrategyFactory;
|
||||
import com.actiontech.dble.route.parser.druid.DruidShardingParseInfo;
|
||||
import com.actiontech.dble.route.parser.druid.RouteCalculateUnit;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.alibaba.druid.sql.ast.SQLStatement;
|
||||
import com.alibaba.druid.sql.dialect.mysql.parser.MySqlStatementParser;
|
||||
import com.alibaba.druid.sql.parser.SQLStatementParser;
|
||||
import com.alibaba.druid.sql.visitor.SchemaStatVisitor;
|
||||
import com.alibaba.druid.stat.TableStat.Condition;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.lang.reflect.Method;
|
||||
import java.sql.SQLSyntaxErrorException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@Ignore
|
||||
public class DQLRouteTest {
|
||||
|
||||
protected Map<String, SchemaConfig> schemaMap;
|
||||
protected LayerCachePool cachePool = new SimpleCachePool();
|
||||
protected RouteStrategy routeStrategy = RouteStrategyFactory.getRouteStrategy();
|
||||
private Map<String, String> tableAliasMap = new HashMap<String, String>();
|
||||
|
||||
protected DruidShardingParseInfo ctx;
|
||||
|
||||
public DQLRouteTest() {
|
||||
String schemaFile = "/route/schema.xml";
|
||||
String ruleFile = "/route/rule.xml";
|
||||
SchemaLoader schemaLoader = new XMLSchemaLoader(schemaFile, ruleFile, true, null);
|
||||
schemaMap = schemaLoader.getSchemas();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test() throws Exception {
|
||||
String stmt = "select * from `offer` where id = 100";
|
||||
SchemaConfig schema = schemaMap.get("mysqldb");
|
||||
RouteResultset rrs = new RouteResultset(stmt, 7);
|
||||
SQLStatementParser parser = null;
|
||||
|
||||
parser = new MySqlStatementParser(stmt);
|
||||
SQLStatement statement;
|
||||
ServerSchemaStatVisitor visitor = null;
|
||||
|
||||
try {
|
||||
statement = parser.parseStatement();
|
||||
visitor = new ServerSchemaStatVisitor();
|
||||
} catch (Exception t) {
|
||||
throw new SQLSyntaxErrorException(t);
|
||||
}
|
||||
ctx = new DruidShardingParseInfo();
|
||||
|
||||
List<RouteCalculateUnit> taskList = visitorParse(rrs, statement, visitor);
|
||||
Assert.assertEquals(true, !taskList.get(0).getTablesAndConditions().isEmpty());
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private List<RouteCalculateUnit> visitorParse(RouteResultset rrs, SQLStatement stmt, ServerSchemaStatVisitor visitor) throws Exception {
|
||||
|
||||
stmt.accept(visitor);
|
||||
|
||||
List<List<Condition>> mergedConditionList = visitor.getConditionList();
|
||||
|
||||
if (visitor.getAliasMap() != null) {
|
||||
for (Map.Entry<String, String> entry : visitor.getAliasMap().entrySet()) {
|
||||
String key = entry.getKey();
|
||||
String value = entry.getValue();
|
||||
if (key != null && key.indexOf("`") >= 0) {
|
||||
key = key.replaceAll("`", "");
|
||||
}
|
||||
if (value != null && value.indexOf("`") >= 0) {
|
||||
value = value.replaceAll("`", "");
|
||||
}
|
||||
// remove the database of table
|
||||
if (key != null) {
|
||||
int pos = key.indexOf(".");
|
||||
if (pos > 0) {
|
||||
key = key.substring(pos + 1);
|
||||
}
|
||||
}
|
||||
|
||||
if (key.equals(value)) {
|
||||
ctx.addTable(key.toUpperCase());
|
||||
}
|
||||
// else {
|
||||
// tableAliasMap.put(key, value);
|
||||
// }
|
||||
tableAliasMap.put(key.toUpperCase(), value);
|
||||
}
|
||||
visitor.getAliasMap().putAll(tableAliasMap);
|
||||
ctx.setTableAliasMap(tableAliasMap);
|
||||
}
|
||||
|
||||
Class<?> clazz = Class.forName("com.actiontech.dble.route.parser.druid.impl.DefaultDruidParser");
|
||||
Method buildRouteCalculateUnits = clazz.getDeclaredMethod("buildRouteCalculateUnits",
|
||||
new Class[]{SchemaStatVisitor.class, List.class});
|
||||
//System.out.println("buildRouteCalculateUnits:\t" + buildRouteCalculateUnits);
|
||||
Object newInstance = clazz.newInstance();
|
||||
buildRouteCalculateUnits.setAccessible(true);
|
||||
Object returnValue = buildRouteCalculateUnits.invoke(newInstance,
|
||||
new Object[]{visitor, mergedConditionList});
|
||||
List<RouteCalculateUnit> retList = new ArrayList<RouteCalculateUnit>();
|
||||
if (returnValue instanceof ArrayList<?>) {
|
||||
retList.add(((ArrayList<RouteCalculateUnit>) returnValue).get(0));
|
||||
//retList = (ArrayList<RouteCalculateUnit>)returnValue;
|
||||
//System.out.println(taskList.get(0).getTablesAndConditions().values());
|
||||
}
|
||||
return retList;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -21,7 +21,7 @@ import com.alibaba.druid.sql.ast.statement.SQLCreateTableStatement;
|
||||
import com.alibaba.druid.sql.ast.statement.SQLTableElement;
|
||||
import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlInsertStatement;
|
||||
import com.alibaba.druid.sql.dialect.mysql.parser.MySqlStatementParser;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
@@ -14,7 +14,7 @@ import com.actiontech.dble.route.factory.RouteStrategyFactory;
|
||||
import com.actiontech.dble.server.parser.ServerParse;
|
||||
import com.alibaba.druid.sql.ast.SQLStatement;
|
||||
import com.alibaba.druid.sql.dialect.mysql.parser.MySqlStatementParser;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import junit.framework.TestCase;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
@@ -56,7 +56,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
RouteResultset rrs = routeStrategy.route(schema, -1, sql,
|
||||
null, cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals("detail_dn15", rrs.getNodes()[0].getName());
|
||||
Assert.assertEquals(
|
||||
@@ -75,7 +75,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
schema = schemaMap.get("cndb");
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals("detail_dn15", rrs.getNodes()[0].getName());
|
||||
Assert.assertEquals(
|
||||
@@ -86,7 +86,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
schema = schemaMap.get("cndb");
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals("offer_dn12", rrs.getNodes()[0].getName());
|
||||
Assert.assertEquals(
|
||||
@@ -116,27 +116,27 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
schema = schemaMap.get("TESTDB");
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
// query of global table only route to one datanode
|
||||
sql = "insert into company (id,name,level) values(111,'company1',3)";
|
||||
schema = schemaMap.get("TESTDB");
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(3, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
|
||||
// update of global table route to every datanode defined
|
||||
sql = "update company set name=name+aaa";
|
||||
schema = schemaMap.get("TESTDB");
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(3, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
|
||||
// delete of global table route to every datanode defined
|
||||
sql = "delete from company where id = 1";
|
||||
schema = schemaMap.get("TESTDB");
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(3, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
|
||||
// company is global table ,will route to differnt tables
|
||||
schema = schemaMap.get("TESTDB");
|
||||
@@ -144,7 +144,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
Set<String> nodeSet = new HashSet<String>();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
nodeSet.add(rrs.getNodes()[0].getName());
|
||||
|
||||
@@ -162,7 +162,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
schema = schemaMap.get("TESTDB");
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
|
||||
}
|
||||
|
||||
@@ -173,7 +173,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
RouteResultset rrs = routeStrategy.route(schema, -1, sql,
|
||||
null, cachePool);
|
||||
Assert.assertEquals(2, rrs.getNodes().length);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(10, rrs.getLimitSize());
|
||||
Assert.assertEquals("dn1", rrs.getNodes()[0].getName());
|
||||
Assert.assertEquals("dn2", rrs.getNodes()[1].getName());
|
||||
@@ -189,7 +189,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
RouteResultset rrs = routeStrategy.route(schema, -1, sql,
|
||||
null, cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1, rrs.getLimitSize());
|
||||
Assert.assertEquals("dn2", rrs.getNodes()[0].getName());
|
||||
|
||||
@@ -197,21 +197,21 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "select * from employee where id=89";
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(2, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1, rrs.getLimitSize());
|
||||
|
||||
// update cache ID found
|
||||
sql = "update employee set name='aaa' where id=88";
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("dn2", rrs.getNodes()[0].getName());
|
||||
|
||||
// delete cache ID found
|
||||
sql = "delete from employee where id=88";
|
||||
rrs = routeStrategy.route(schema, -1, sql, null, cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("dn2", rrs.getNodes()[0].getName());
|
||||
|
||||
}
|
||||
@@ -331,7 +331,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
String sql = "select * from independent where member='abc'";
|
||||
RouteResultset rrs = routeStrategy.route(schema, 1, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Map<String, RouteResultsetNode> nodeMap = getNodeMap(rrs, 128);
|
||||
IndexedNodeNameAsserter nameAsserter = new IndexedNodeNameAsserter(
|
||||
"independent_dn", 0, 128);
|
||||
@@ -351,7 +351,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "select * from cndb.independent A where a.member='abc'";
|
||||
schema = schemaMap.get("cndb");
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
nodeMap = getNodeMap(rrs, 128);
|
||||
nameAsserter = new IndexedNodeNameAsserter("independent_dn", 0, 128);
|
||||
nameAsserter.assertRouteNodeNames(nodeMap.keySet());
|
||||
@@ -373,12 +373,12 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
RouteResultset rrs = routeStrategy.route(schema, 1, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("dn1", rrs.getNodes()[0].getName());
|
||||
|
||||
sql = "insert into orders (id,name,customer_id) values(1,'testonly',2000001)";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals("dn2", rrs.getNodes()[0].getName());
|
||||
|
||||
@@ -397,7 +397,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
// route by parent rule ,update sql
|
||||
sql = "update orders set id=1 ,name='aaa' where customer_id=2000001";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("dn2", rrs.getNodes()[0].getName());
|
||||
|
||||
// route by parent rule but can't find datanode
|
||||
@@ -413,7 +413,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
// route by parent rule ,select sql
|
||||
sql = "select * from orders where customer_id=2000001";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("dn2", rrs.getNodes()[0].getName());
|
||||
|
||||
// route by parent rule ,delete sql
|
||||
@@ -438,7 +438,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "select * from cndb.offer where (offer_id, group_id ) In (123,234)";
|
||||
schema = schemaMap.get("cndb");
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(128, rrs.getNodes().length);
|
||||
for (int i = 0; i < 128; i++) {
|
||||
@@ -450,7 +450,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "SELECT * FROM offer WHERE FALSE OR offer_id = 123 AND member_id = 123 OR member_id = 123 AND member_id = 234 OR member_id = 123 AND member_id = 345 OR member_id = 123 AND member_id = 456 OR offer_id = 234 AND group_id = 123 OR offer_id = 234 AND group_id = 234 OR offer_id = 234 AND group_id = 345 OR offer_id = 234 AND group_id = 456 OR offer_id = 345 AND group_id = 123 OR offer_id = 345 AND group_id = 234 OR offer_id = 345 AND group_id = 345 OR offer_id = 345 AND group_id = 456 OR offer_id = 456 AND group_id = 123 OR offer_id = 456 AND group_id = 234 OR offer_id = 456 AND group_id = 345 OR offer_id = 456 AND group_id = 456";
|
||||
schema = schemaMap.get("cndb");
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
getNodeMap(rrs, 128);
|
||||
|
||||
sql = "select * from offer where false"
|
||||
@@ -460,7 +460,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
+ " or offer_id=123 and group_id=456 ";
|
||||
schema = schemaMap.get("cndb");
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
|
||||
}
|
||||
@@ -473,7 +473,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
|
||||
sql = "select * from orders";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SELECT, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Map<String, RouteResultsetNode> nodeMap = getNodeMap(rrs, 2);
|
||||
NodeNameAsserter nameAsserter = new NodeNameAsserter("dn2",
|
||||
"dn1");
|
||||
@@ -487,7 +487,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
|
||||
sql = "select * from goods";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SELECT, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(schema.getDefaultMaxLimit(), rrs.getLimitSize());
|
||||
// Assert.assertEquals("select * from goods", rrs.getNodes()[0].getStatement());
|
||||
@@ -498,7 +498,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
|
||||
sql = "select * from goods limit 2 ,3";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SELECT, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// Assert.assertEquals(-1, rrs.getLimitSize());
|
||||
Assert.assertEquals("select * from goods limit 2 ,3", rrs.getNodes()[0].getStatement());
|
||||
@@ -506,7 +506,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
|
||||
sql = "select * from notpartionTable limit 2 ,3";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SELECT, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals(3, rrs.getLimitSize());
|
||||
Assert.assertEquals("select * from notpartionTable limit 2 ,3", rrs.getNodes()[0].getStatement());
|
||||
@@ -522,7 +522,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
//SQL span multi datanode
|
||||
sql = "select * from orders limit 2,3";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SELECT, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
Map<String, RouteResultsetNode> nodeMap = getNodeMap(rrs, 2);
|
||||
NodeNameAsserter nameAsserter = new NodeNameAsserter("dn2",
|
||||
"dn1");
|
||||
@@ -538,7 +538,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
//SQL not span multi datanode
|
||||
sql = "select * from customer where id=10000 limit 2,3";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SELECT, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
nodeMap = getNodeMap(rrs, 1);
|
||||
nameAsserter = new NodeNameAsserter("dn1");
|
||||
nameAsserter.assertRouteNodeNames(nodeMap.keySet());
|
||||
@@ -557,7 +557,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "select count(*) from (select * from(select * from offer_detail where offer_id='123' or offer_id='234' limit 88)offer where offer.member_id='abc' limit 60) w "
|
||||
+ " where w.member_id ='pavarotti17' limit 99";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
// Assert.assertEquals(88L, rrs.getLimitSize());
|
||||
// Assert.assertEquals(RouteResultset.SUM_FLAG, rrs.getFlag());
|
||||
Map<String, RouteResultsetNode> nodeMap = getNodeMap(rrs, 2);
|
||||
@@ -568,7 +568,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "select count(*) from (select * from(select max(id) from offer_detail where offer_id='123' or offer_id='234' limit 88)offer where offer.member_id='abc' limit 60) w "
|
||||
+ " where w.member_id ='pavarotti17' limit 99";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
nodeMap = getNodeMap(rrs, 2);
|
||||
nameAsserter = new NodeNameAsserter("detail_dn29", "detail_dn15");
|
||||
nameAsserter.assertRouteNodeNames(nodeMap.keySet());
|
||||
@@ -576,7 +576,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "select * from (select * from(select max(id) from offer_detail where offer_id='123' or offer_id='234' limit 88)offer where offer.member_id='abc' limit 60) w "
|
||||
+ " where w.member_id ='pavarotti17' limit 99";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
nodeMap = getNodeMap(rrs, 2);
|
||||
nameAsserter = new NodeNameAsserter("detail_dn29", "detail_dn15");
|
||||
nameAsserter.assertRouteNodeNames(nodeMap.keySet());
|
||||
@@ -584,7 +584,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "select * from (select count(*) from(select * from offer_detail where offer_id='123' or offer_id='234' limit 88)offer where offer.member_id='abc' limit 60) w "
|
||||
+ " where w.member_id ='pavarotti17' limit 99";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(true, rrs.isCacheAble());
|
||||
Assert.assertEquals(true, rrs.isSqlRouteCacheAble());
|
||||
// Assert.assertEquals(88L, rrs.getLimitSize());
|
||||
// Assert.assertEquals(RouteResultset.SUM_FLAG, rrs.getFlag());
|
||||
nodeMap = getNodeMap(rrs, 2);
|
||||
@@ -599,7 +599,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
String sql = " desc offer";
|
||||
RouteResultset rrs = routeStrategy.route(schema, ServerParse.DESCRIBE, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -608,7 +608,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
|
||||
sql = "desc cndb.offer";
|
||||
rrs = routeStrategy.route(schema, ServerParse.DESCRIBE, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -617,7 +617,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
|
||||
sql = "desc cndb.offer col1";
|
||||
rrs = routeStrategy.route(schema, ServerParse.DESCRIBE, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -627,7 +627,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "SHOW FULL COLUMNS FROM offer IN db_name WHERE true";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -639,7 +639,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
// Assert.assertEquals("offer_dn[0]", rrs.getNodes()[0].getName());
|
||||
@@ -651,13 +651,13 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("SHOW FULL TABLES WHERE Table_type != 'VIEW'", rrs.getNodes()[0].getStatement());
|
||||
|
||||
sql = "SHOW INDEX IN offer FROM db_name";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
// random return one node
|
||||
@@ -668,7 +668,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "SHOW TABLES from db_name like 'solo'";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Map<String, RouteResultsetNode> nodeMap = getNodeMap(rrs, 3);
|
||||
NodeNameAsserter nameAsserter = new NodeNameAsserter("detail_dn0",
|
||||
@@ -688,7 +688,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "SHOW TABLES in db_name ";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
nodeMap = getNodeMap(rrs, 3);
|
||||
nameAsserter = new NodeNameAsserter("detail_dn0", "offer_dn0",
|
||||
@@ -706,7 +706,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
sql = "SHOW TABLeS ";
|
||||
rrs = routeStrategy.route(schema, ServerParse.SHOW, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
nodeMap = getNodeMap(rrs, 3);
|
||||
nameAsserter = new NodeNameAsserter("offer_dn0", "detail_dn0",
|
||||
@@ -751,21 +751,21 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
String sql = "select * from offer where offer_id=1";
|
||||
RouteResultset rrs = routeStrategy.route(schema, 1, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("cndb_dn", rrs.getNodes()[0].getName());
|
||||
Assert.assertEquals(sql, rrs.getNodes()[0].getStatement());
|
||||
sql = "select * from ignoreSchemaTest.offer1 where ignoreSchemaTest.offer1.offer_id=1";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("select * from offer1 where offer1.offer_id=1",
|
||||
rrs.getNodes()[0].getStatement());
|
||||
sql = "select * from ignoreSchemaTest2.offer where ignoreSchemaTest2.offer.offer_id=1";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(sql, rrs.getNodes()[0].getStatement(), sql);
|
||||
sql = "select * from ignoreSchemaTest2.offer a,offer b where ignoreSchemaTest2.offer.offer_id=1";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(
|
||||
"select * from ignoreSchemaTest2.offer a,offer b where ignoreSchemaTest2.offer.offer_id=1",
|
||||
rrs.getNodes()[0].getStatement());
|
||||
@@ -781,7 +781,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
schema = schemaMap.get("dubbo2");
|
||||
sql = "SHOW TABLES from db_name like 'solo'";
|
||||
rrs = routeStrategy.route(schema, 9, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals("dn1", rrs.getNodes()[0].getName());
|
||||
@@ -791,7 +791,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
schema = schemaMap.get("dubbo");
|
||||
sql = "SHOW TABLES from db_name like 'solo'";
|
||||
rrs = routeStrategy.route(schema, 9, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals("dubbo_dn", rrs.getNodes()[0].getName());
|
||||
@@ -801,7 +801,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
|
||||
sql = "desc cndb.offer";
|
||||
rrs = routeStrategy.route(schema, 1, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals(-1L, rrs.getLimitSize());
|
||||
Assert.assertEquals(1, rrs.getNodes().length);
|
||||
Assert.assertEquals("dubbo_dn", rrs.getNodes()[0].getName());
|
||||
@@ -810,7 +810,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
schema = schemaMap.get("cndb");
|
||||
sql = "SHOW fulL TaBLES from db_name like 'solo'";
|
||||
rrs = routeStrategy.route(schema, 9, sql, null, cachePool);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Map<String, RouteResultsetNode> nodeMap = getNodeMap(rrs, 3);
|
||||
NodeNameAsserter nameAsserter = new NodeNameAsserter("detail_dn0",
|
||||
"offer_dn0", "independent_dn0");
|
||||
@@ -888,7 +888,7 @@ public class DruidMysqlRouteStrategyTest extends TestCase {
|
||||
rrs = routeStrategy.route(schema, 1, sql, null,
|
||||
cachePool);
|
||||
Assert.assertEquals(2, rrs.getNodes().length);
|
||||
Assert.assertEquals(false, rrs.isCacheAble());
|
||||
Assert.assertEquals(false, rrs.isSqlRouteCacheAble());
|
||||
Assert.assertEquals("dn1", rrs.getNodes()[0].getName());
|
||||
Assert.assertEquals("dn2", rrs.getNodes()[1].getName());
|
||||
String node1Sql = formatSql("insert into employee (id,name,sharding_id) values(1,'testonly',10000)");
|
||||
|
||||
@@ -12,7 +12,7 @@ import com.actiontech.dble.config.loader.xml.XMLSchemaLoader;
|
||||
import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.route.factory.RouteStrategyFactory;
|
||||
import com.actiontech.dble.server.parser.ServerParse;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
@@ -1,293 +0,0 @@
|
||||
package com.actiontech.dble.route;
|
||||
|
||||
import com.actiontech.dble.route.impl.DefaultRouteStrategy;
|
||||
import com.actiontech.dble.route.parser.druid.RouteCalculateUnit;
|
||||
import com.actiontech.dble.route.parser.druid.ServerSchemaStatVisitor;
|
||||
import com.actiontech.dble.route.parser.druid.impl.DefaultDruidParser;
|
||||
import com.alibaba.druid.sql.ast.SQLStatement;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Created by szf on 2018/12/19.
|
||||
*/
|
||||
public class RouteWithOrTest {
|
||||
|
||||
@Test
|
||||
public void test() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest where (name = 'bbb' or name = 'ccc') and id = 12");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test1() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest where (name = 'bbb' or (name = 'ccc' or name ='ddd')) and id =12");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(2);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test3() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest where ((name = 'bbb' or name = 'ccc') or (name = 'eee' or (name = 'fff' or name ='ggg'))) and id = 12");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(2);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(3);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(4);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void test4() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest where (name = 'bbb' or name = 'ccc') and (id = 12 or id =14)");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(2);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(3);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test5() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest where (name = 'bbb' or name = 'ccc') and id = 12 and pk=11");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("PK"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("PK"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test6() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest where (name = 'bbb' or name = 'ccc') and (id = 12 OR ID = 13) and pk=11");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("PK"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("PK"));
|
||||
unit = result.get(2);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("PK"));
|
||||
unit = result.get(3);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("PK"));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void test7() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest where (name = 'bbb' or name = 'ccc') and (id = 12 OR id = 13) OR pk=11");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(1, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("PK"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(2);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(3);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(4);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void test8() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest,mtest where (ntest.name = 'cccc' or ntest.name = 'bbbb') and ntest.id = mtest.id and mtest.id = 1 ");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(1, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(1, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void test9() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest,mtest where ntest.id = mtest.id and mtest.id = 1 and (ntest.name = 'cccc' or ntest.name = 'bbbb') and (mtest.name = 'aaaa' or mtest.name = 'dddd')");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(2);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
unit = result.get(3);
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void test10() {
|
||||
List<RouteCalculateUnit> result = getCoreCalculateUnit("select * from ntest,mtest,xtest where ntest.id = mtest.id and xtest.id = ntest.id and mtest.id = 1 and (ntest.name = 'cccc' or ntest.name = 'bbbb') and (mtest.name = 'aaaa' or mtest.name = 'dddd')");
|
||||
RouteCalculateUnit unit = result.get(0);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(1, unit.getTablesAndConditions().get("xtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("xtest").containsKey("ID"));
|
||||
unit = result.get(1);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(1, unit.getTablesAndConditions().get("xtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("xtest").containsKey("ID"));
|
||||
unit = result.get(2);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(1, unit.getTablesAndConditions().get("xtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("xtest").containsKey("ID"));
|
||||
unit = result.get(3);
|
||||
Assert.assertEquals(3, unit.getTablesAndConditions().size());
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("ntest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(2, unit.getTablesAndConditions().get("mtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("ID"));
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("ntest").containsKey("NAME"));
|
||||
Assert.assertEquals(1, unit.getTablesAndConditions().get("xtest").size());
|
||||
Assert.assertTrue(unit.getTablesAndConditions().get("xtest").containsKey("ID"));
|
||||
|
||||
}
|
||||
|
||||
|
||||
public List<RouteCalculateUnit> getCoreCalculateUnit(String sql) throws RuntimeException {
|
||||
try {
|
||||
ServerSchemaStatVisitor visitor = new ServerSchemaStatVisitor();
|
||||
DefaultRouteStrategy x = new DefaultRouteStrategy();
|
||||
SQLStatement statement = x.parserSQL(sql);
|
||||
statement.accept(visitor);
|
||||
|
||||
DefaultDruidParser parser = new DefaultDruidParser();
|
||||
Class class1 = parser.getClass();
|
||||
Method format = class1.getDeclaredMethod("getTableAliasMap", Map.class);
|
||||
format.setAccessible(true);
|
||||
Map<String, String> tableAliasMap = (Map<String, String>) format.invoke(parser, visitor.getAliasMap());
|
||||
|
||||
Method format2 = class1.getDeclaredMethod("buildRouteCalculateUnits", Map.class, List.class);
|
||||
format2.setAccessible(true);
|
||||
List<RouteCalculateUnit> result = (List<RouteCalculateUnit>) format2.invoke(parser, tableAliasMap, visitor.getConditionList());
|
||||
return result;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
@@ -11,7 +11,7 @@ import com.actiontech.dble.config.loader.SchemaLoader;
|
||||
import com.actiontech.dble.config.loader.xml.XMLSchemaLoader;
|
||||
import com.actiontech.dble.config.model.SchemaConfig;
|
||||
import com.actiontech.dble.route.factory.RouteStrategyFactory;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
@@ -7,7 +7,7 @@ package com.actiontech.dble.sequence;
|
||||
|
||||
import com.actiontech.dble.route.sequence.handler.IncrSequenceZKHandler;
|
||||
import com.actiontech.dble.route.util.PropertiesUtil;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.apache.curator.test.TestingServer;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
@@ -5,7 +5,7 @@
|
||||
*/
|
||||
package com.actiontech.dble.sqlexecute;
|
||||
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
|
||||
import java.sql.*;
|
||||
|
||||
|
||||
@@ -5,7 +5,7 @@
|
||||
*/
|
||||
package com.actiontech.dble.util;
|
||||
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
@@ -5,7 +5,7 @@
|
||||
*/
|
||||
package com.actiontech.dble.util;
|
||||
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user