diff --git a/src/main/java/com/actiontech/dble/DbleServer.java b/src/main/java/com/actiontech/dble/DbleServer.java index 3951c4efb..59ce2a91c 100644 --- a/src/main/java/com/actiontech/dble/DbleServer.java +++ b/src/main/java/com/actiontech/dble/DbleServer.java @@ -942,10 +942,10 @@ public final class DbleServer { public void run() { Map 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(); } } }; diff --git a/src/main/java/com/actiontech/dble/backend/datasource/PhysicalDatasource.java b/src/main/java/com/actiontech/dble/backend/datasource/PhysicalDatasource.java index 472ef22f1..72049dbc1 100644 --- a/src/main/java/com/actiontech/dble/backend/datasource/PhysicalDatasource.java +++ b/src/main/java/com/actiontech/dble/backend/datasource/PhysicalDatasource.java @@ -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; diff --git a/src/main/java/com/actiontech/dble/backend/mysql/VersionUtil.java b/src/main/java/com/actiontech/dble/backend/mysql/VersionUtil.java new file mode 100644 index 000000000..4cddc7f48 --- /dev/null +++ b/src/main/java/com/actiontech/dble/backend/mysql/VersionUtil.java @@ -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; + } + } +} diff --git a/src/main/java/com/actiontech/dble/backend/mysql/nio/MySQLDataSource.java b/src/main/java/com/actiontech/dble/backend/mysql/nio/MySQLDataSource.java index 3f92667d7..7ac5de738 100644 --- a/src/main/java/com/actiontech/dble/backend/mysql/nio/MySQLDataSource.java +++ b/src/main/java/com/actiontech/dble/backend/mysql/nio/MySQLDataSource.java @@ -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(); diff --git a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/BaseHandlerBuilder.java b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/BaseHandlerBuilder.java index 7d5b05000..b297fdc44 100644 --- a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/BaseHandlerBuilder.java +++ b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/BaseHandlerBuilder.java @@ -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 schemaConfigMap = new HashMap<>(); + Map 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 subQueryBuilderList = new CopyOnWriteArrayList<>(); + private List 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 mapTableToSimple = visitor.getMapTableToSimple(); + for (Map.Entry 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 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 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 orderBys) { List childOrders = qn.getChild().getOrderBys(); List 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 mergeOrderBy(List columnsSelected, List orderBys) { List 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 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 dataNodes) { + String getRandomNode(Set 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; } diff --git a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/JoinNodeHandlerBuilder.java b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/JoinNodeHandlerBuilder.java index 105c8fc98..130ff2a37 100644 --- a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/JoinNodeHandlerBuilder.java +++ b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/JoinNodeHandlerBuilder.java @@ -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 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 valueSet, int maxPartSize) { + private void buildNestFilters(PlanNode tnBig, Item keyToPass, Set valueSet, int maxPartSize) { List strategyFilters = tnBig.getNestLoopFilters(); List 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 argList = new ArrayList<>(); diff --git a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/MergeBuilder.java b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/MergeBuilder.java index c8191f145..9d65f309d 100644 --- a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/MergeBuilder.java +++ b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/MergeBuilder.java @@ -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 schemaConfigMap = new HashMap<>(); + private Map 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 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 mapTableToSimple, SQLSelectStatement select, SchemaConfig schemaConfig) throws SQLException { RouteResultset rrs = new RouteResultset(sql, ServerParse.SELECT); + String pushDownSQL = rrs.getStatement(); + for (Map.Entry tableToSimple : mapTableToSimple.entrySet()) { + pushDownSQL = pushDownSQL.replace(tableToSimple.getKey(), tableToSimple.getValue()); + } + rrs.setStatement(pushDownSQL); LayerCachePool pool = DbleServer.getInstance().getRouterService().getTableId2DataNodeCache(); - Map tableConfigMap = new HashMap<>(); + Map, 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 -------------------- */ diff --git a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/TableNodeHandlerBuilder.java b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/TableNodeHandlerBuilder.java index b5132c302..e515aa260 100644 --- a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/TableNodeHandlerBuilder.java +++ b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/TableNodeHandlerBuilder.java @@ -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 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 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) { diff --git a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/GlobalVisitor.java b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/GlobalVisitor.java index 268a97c59..b8f5ab320 100644 --- a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/GlobalVisitor.java +++ b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/GlobalVisitor.java @@ -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(); } } diff --git a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/MysqlVisitor.java b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/MysqlVisitor.java index 6b453cd6a..8240b82d0 100644 --- a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/MysqlVisitor.java +++ b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/MysqlVisitor.java @@ -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 pushNameMap = new HashMap<>(); - protected boolean isTopQuery = false; + Map 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 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 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(); } diff --git a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/PushDownVisitor.java b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/PushDownVisitor.java index 5de370dfd..4f312f185 100644 --- a/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/PushDownVisitor.java +++ b/src/main/java/com/actiontech/dble/backend/mysql/nio/handler/builder/sqlvisitor/PushDownVisitor.java @@ -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) { diff --git a/src/main/java/com/actiontech/dble/config/ConfigInitializer.java b/src/main/java/com/actiontech/dble/config/ConfigInitializer.java index c90c09d0a..eec5dd1fe 100644 --- a/src/main/java/com/actiontech/dble/config/ConfigInitializer.java +++ b/src/main/java/com/actiontech/dble/config/ConfigInitializer.java @@ -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); diff --git a/src/main/java/com/actiontech/dble/manager/handler/SelectHandler.java b/src/main/java/com/actiontech/dble/manager/handler/SelectHandler.java index c01150501..bb3987bb5 100644 --- a/src/main/java/com/actiontech/dble/manager/handler/SelectHandler.java +++ b/src/main/java/com/actiontech/dble/manager/handler/SelectHandler.java @@ -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); diff --git a/src/main/java/com/actiontech/dble/manager/response/ReloadSqlSlowTime.java b/src/main/java/com/actiontech/dble/manager/response/ReloadSqlSlowTime.java index 7ebc56ad0..35e2fac4f 100644 --- a/src/main/java/com/actiontech/dble/manager/response/ReloadSqlSlowTime.java +++ b/src/main/java/com/actiontech/dble/manager/response/ReloadSqlSlowTime.java @@ -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 statMap = UserStatAnalyzer.getInstance().getUserStatMap(); - for (UserStat userStat : statMap.values()) { - userStat.setSlowTime(time); - } + //Map statMap = UserStatAnalyzer.getInstance().getUserStatMap(); + //for (UserStat userStat : statMap.values()) { + // userStat.setSlowTime(time); + //} LOGGER.info(String.valueOf(c) + " reload @@sqlslow=" + time + " success by manager"); diff --git a/src/main/java/com/actiontech/dble/manager/response/SelectMaxAllowedPacket.java b/src/main/java/com/actiontech/dble/manager/response/SelectMaxAllowedPacket.java index 58c7459a3..bc7bd9fce 100644 --- a/src/main/java/com/actiontech/dble/manager/response/SelectMaxAllowedPacket.java +++ b/src/main/java/com/actiontech/dble/manager/response/SelectMaxAllowedPacket.java @@ -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); diff --git a/src/main/java/com/actiontech/dble/manager/response/SelectSessionTxReadOnly.java b/src/main/java/com/actiontech/dble/manager/response/SelectSessionTxReadOnly.java index 44ecc2464..d113a26f9 100644 --- a/src/main/java/com/actiontech/dble/manager/response/SelectSessionTxReadOnly.java +++ b/src/main/java/com/actiontech/dble/manager/response/SelectSessionTxReadOnly.java @@ -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); diff --git a/src/main/java/com/actiontech/dble/manager/response/ShowSQLHigh.java b/src/main/java/com/actiontech/dble/manager/response/ShowSQLHigh.java index 6f73a10bb..1a4bbdbc3 100644 --- a/src/main/java/com/actiontech/dble/manager/response/ShowSQLHigh.java +++ b/src/main/java/com/actiontech/dble/manager/response/ShowSQLHigh.java @@ -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; } diff --git a/src/main/java/com/actiontech/dble/net/mysql/HandshakeV10Packet.java b/src/main/java/com/actiontech/dble/net/mysql/HandshakeV10Packet.java old mode 100755 new mode 100644 index f4638a30e..5d5e1f207 --- a/src/main/java/com/actiontech/dble/net/mysql/HandshakeV10Packet.java +++ b/src/main/java/com/actiontech/dble/net/mysql/HandshakeV10Packet.java @@ -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()); diff --git a/src/main/java/com/actiontech/dble/plan/common/item/function/operator/cmpfunc/ItemFuncOptNeg.java b/src/main/java/com/actiontech/dble/plan/common/item/function/operator/cmpfunc/ItemFuncOptNeg.java index 1a247a58b..f17664c9e 100644 --- a/src/main/java/com/actiontech/dble/plan/common/item/function/operator/cmpfunc/ItemFuncOptNeg.java +++ b/src/main/java/com/actiontech/dble/plan/common/item/function/operator/cmpfunc/ItemFuncOptNeg.java @@ -34,4 +34,8 @@ public abstract class ItemFuncOptNeg extends ItemIntFunc { negated = !negated; } + public boolean isNegate() { + return negated; + } + } diff --git a/src/main/java/com/actiontech/dble/route/RouteResultset.java b/src/main/java/com/actiontech/dble/route/RouteResultset.java index a091b9f3a..6fd174ef0 100644 --- a/src/main/java/com/actiontech/dble/route/RouteResultset.java +++ b/src/main/java/com/actiontech/dble/route/RouteResultset.java @@ -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() { diff --git a/src/main/java/com/actiontech/dble/route/RouteService.java b/src/main/java/com/actiontech/dble/route/RouteService.java index 7ffd7ca59..7ad788271 100644 --- a/src/main/java/com/actiontech/dble/route/RouteService.java +++ b/src/main/java/com/actiontech/dble/route/RouteService.java @@ -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); } diff --git a/src/main/java/com/actiontech/dble/route/impl/AbstractRouteStrategy.java b/src/main/java/com/actiontech/dble/route/impl/AbstractRouteStrategy.java index f564947f5..cfadbe2ed 100644 --- a/src/main/java/com/actiontech/dble/route/impl/AbstractRouteStrategy.java +++ b/src/main/java/com/actiontech/dble/route/impl/AbstractRouteStrategy.java @@ -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) { diff --git a/src/main/java/com/actiontech/dble/route/parser/ManagerParseSelect.java b/src/main/java/com/actiontech/dble/route/parser/ManagerParseSelect.java index 2220437e8..fc7a126d8 100644 --- a/src/main/java/com/actiontech/dble/route/parser/ManagerParseSelect.java +++ b/src/main/java/com/actiontech/dble/route/parser/ManagerParseSelect.java @@ -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; + } + } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/DruidShardingParseInfo.java b/src/main/java/com/actiontech/dble/route/parser/druid/DruidShardingParseInfo.java index b95179cf7..197957baa 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/DruidShardingParseInfo.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/DruidShardingParseInfo.java @@ -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 routeCalculateUnits = new ArrayList<>(); - private List tables = new ArrayList<>(); + private List> tables = new ArrayList<>(); /** - * key table alias, value talbe realname; + * key table alias, value table real name; */ private Map tableAliasMap = new LinkedHashMap<>(); @@ -34,17 +36,14 @@ public class DruidShardingParseInfo { this.tableAliasMap = tableAliasMap; } - public List getTables() { + public List> getTables() { return tables; } - public void addTable(String tableName) { - this.tables.add(tableName); + public void addTable(Pair table) { + this.tables.add(table); } - public RouteCalculateUnit getRouteCalculateUnit() { - return routeCalculateUnits.get(0); - } public List 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(); } - } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/RouteCalculateUnit.java b/src/main/java/com/actiontech/dble/route/parser/druid/RouteCalculateUnit.java index 4297ec7c2..374bc12bf 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/RouteCalculateUnit.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/RouteCalculateUnit.java @@ -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>> tablesAndConditions = new LinkedHashMap<>(); + private Map, Map>> tablesAndConditions = new LinkedHashMap<>(); - public Map>> getTablesAndConditions() { + public Map, Map>> getTablesAndConditions() { return tablesAndConditions; } - public void addShardingExpr(String tableName, String columnName, Object value) { - Map> tableColumnsMap = tablesAndConditions.get(tableName); + public void addShardingExpr(Pair table, String columnName, Object value) { + Map> 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, Map>> entry : tablesAndConditions.entrySet()) { + Pair table = entry.getKey(); + String schemaName = table.getKey(); + String tableName = table.getValue(); + Map> columnsMap = entry.getValue(); + for (Map.Entry> columns : columnsMap.entrySet()) { + String columnName = columns.getKey(); + Set 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(); + } } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/ServerSchemaStatVisitor.java b/src/main/java/com/actiontech/dble/route/parser/druid/ServerSchemaStatVisitor.java index 8b50019ef..1bef19319 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/ServerSchemaStatVisitor.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/ServerSchemaStatVisitor.java @@ -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 whereUnits = new CopyOnWriteArrayList<>(); - private List storedWhereUnits = new CopyOnWriteArrayList<>(); private boolean notInWhere = false; private List subQueryList = new ArrayList<>(); private Map aliasMap = new LinkedHashMap<>(); private List 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> 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 *

* This function only recursively all child splitedExpr and make them split again * - * @param whereUnitList */ private void loopFindSubOrCondition(List whereUnitList) { List 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 relationships) { HashSet 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 tempSet, Column tmp) { if (tempSet.contains(tmp)) { @@ -663,107 +634,6 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor { return hasOrCondition; } - private List> 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> mergedConditionList = getMergedConditionList(whereUnit.getSubWhereUnit()); - if (whereUnit.getOutConditions().size() > 0) { - for (List aMergedConditionList : mergedConditionList) { - aMergedConditionList.addAll(whereUnit.getOutConditions()); - } - } - if (whereUnit.getOutRelationships().size() > 0) { - for (List aMergedConditionList : mergedConditionList) { - extendConditionsFromRelations(aMergedConditionList, whereUnit.getOutRelationships()); - } - } - whereUnit.getConditionList().addAll(mergedConditionList); - } else if (whereUnit.getSubWhereUnit().size() == 1) { - List> subConditionList = whereUnit.getSubWhereUnit().get(0).getConditionList(); - if (whereUnit.getOutConditions().size() > 0 && subConditionList.size() > 0) { - for (List aSubConditionList : subConditionList) { - aSubConditionList.addAll(whereUnit.getOutConditions()); - } - } - if (whereUnit.getOutRelationships().size() > 0 && subConditionList.size() > 0) { - for (List aSubConditionList : subConditionList) { - extendConditionsFromRelations(aSubConditionList, whereUnit.getOutRelationships()); - } - } - whereUnit.getConditionList().addAll(subConditionList); - } - } else { - //do nothing - } - } - - /** - * merge WhereUnit's condition - * - * @return - */ - private List> getMergedConditionList(List whereUnitList) { - List> 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> merge(List> list1, List> list2) { - if (list1.size() == 0) { - return list2; - } else if (list2.size() == 0) { - return list1; - } - - List> retList = new ArrayList<>(); - for (List aList1 : list1) { - for (List aList2 : list2) { - List listTmp = new ArrayList<>(); - listTmp.addAll(aList1); - listTmp.addAll(aList2); - retList.add(listTmp); - } - } - return retList; - } - public List 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> retList = new ArrayList<>(); List outSideCondition = new ArrayList<>(); outSideCondition.addAll(conditions); @@ -792,22 +661,21 @@ public class ServerSchemaStatVisitor extends MySqlSchemaStatVisitor { Set 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 conds, Set relations) { - List newConds = new ArrayList<>(); - Iterator 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 subConditionList : sub.getOrConditionList()) { + ConditionUtil.extendConditionsFromRelations(subConditionList, whereUnit.getOutRelationships()); + } } } } - conds.addAll(newConds); } - public List> getConditionList() { - if (this.hasOrCondition()) { - return this.splitConditions(); + public List getAllWhereUnit() { + List 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 conds = this.getConditions(); - Set relations = getRelationships(); - extendConditionsFromRelations(conds, relations); - List> 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 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> retList = new ArrayList<>(); + retList.add(conditionList); + whereUnit.setOrConditionList(retList); + whereUnit.addOutRelationships(this.relationships); + return whereUnit; } } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/WhereUnit.java b/src/main/java/com/actiontech/dble/route/parser/druid/WhereUnit.java index a8ba57c22..f907e675e 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/WhereUnit.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/WhereUnit.java @@ -52,11 +52,11 @@ public class WhereUnit { private List splitedExprList = new ArrayList<>(); - private List> conditionList = new ArrayList<>(); + private List> orConditionList = new ArrayList<>(); /** * whereExpris not contains all where condition,consider outConditions */ - private List outConditions = new ArrayList<>(); + private List outAndConditions = new ArrayList<>(); private Set outRelationships = new HashSet<>(); @@ -65,12 +65,12 @@ public class WhereUnit { private boolean finishedParse = false; - public List getOutConditions() { - return outConditions; + public List getOutAndConditions() { + return outAndConditions; } public void addOutConditions(List conditions) { - this.outConditions.addAll(conditions); + this.outAndConditions.addAll(conditions); } public Set getOutRelationships() { @@ -113,12 +113,12 @@ public class WhereUnit { this.splitedExprList.add(splitedExpr); } - public List> getConditionList() { - return conditionList; + public List> getOrConditionList() { + return orConditionList; } - public void setConditionList(List> conditionList) { - this.conditionList = conditionList; + public void setOrConditionList(List> orConditionList) { + this.orConditionList = orConditionList; } public void addSubWhereUnit(WhereUnit whereUnit) { @@ -128,4 +128,63 @@ public class WhereUnit { public List 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 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(); + } } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DefaultDruidParser.java b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DefaultDruidParser.java index 33c813d27..cab027872 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DefaultDruidParser.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DefaultDruidParser.java @@ -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 tableAliasMap = getTableAliasMap(visitor.getAliasMap()); - ctx.setRouteCalculateUnits(this.buildRouteCalculateUnits(tableAliasMap, visitor.getConditionList())); - return schema; + String schemaName = null; + if (schemaConfig != null) { + schemaName = schemaConfig.getName(); + } + Map tableAliasMap = getTableAliasMap(schemaName, visitor.getAliasMap()); + ctx.setRouteCalculateUnits(ConditionUtil.buildRouteCalculateUnits(visitor.getAllWhereUnit(), tableAliasMap, schemaName)); + + return schemaConfig; } - private Map getTableAliasMap(Map originTableAliasMap) { + private Map getTableAliasMap(String defaultSchemaName, Map originTableAliasMap) { if (originTableAliasMap == null) { return null; } - Map tableAliasMap = new HashMap<>(); - tableAliasMap.putAll(originTableAliasMap); + + Map tableAliasMap = new HashMap<>(originTableAliasMap); for (Map.Entry 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> iterator = tableAliasMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry 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 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 buildRouteCalculateUnits(Map tableAliasMap, List> conditionList) { - List retList = new ArrayList<>(); - //find partition column in condition - for (List aConditionList : conditionList) { - RouteCalculateUnit routeCalculateUnit = new RouteCalculateUnit(); - for (Condition condition : aConditionList) { - List 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 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 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); + } + } } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidInsertParser.java b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidInsertParser.java index 36b9ad887..adb531f24 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidInsertParser.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidInsertParser.java @@ -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()); - } + nodeValuesMap.putIfAbsent(nodeIndex, new ArrayList<>()); nodeValuesMap.get(nodeIndex).add(valueClause); } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidInsertReplaceParser.java b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidInsertReplaceParser.java index d53521bc3..32edf2a1a 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidInsertReplaceParser.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidInsertReplaceParser.java @@ -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 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 columnExprList) + int getTableColumns(SchemaInfo schemaInfo, List 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 columnExprList, String partitionColumn) throws SQLNonTransientException { + int getShardingColIndex(SchemaInfo schemaInfo, List 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()); + } + } + }); + } } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidReplaceParser.java b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidReplaceParser.java index debc65db2..0693fb87c 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidReplaceParser.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidReplaceParser.java @@ -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 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()); - } + nodeValuesMap.computeIfAbsent(nodeIndex, k -> new ArrayList<>()); nodeValuesMap.get(nodeIndex).add(valueClause); } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidSelectParser.java b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidSelectParser.java index c55e74d09..551266b2d 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidSelectParser.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidSelectParser.java @@ -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 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>> allConditions = getAllConditions(); + Map, Map>> 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 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>> getAllConditions() { - Map>> map = new HashMap<>(); + private Map, Map>> getAllConditions() { + Map, Map>> 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 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>> allConditions) { + MySqlSelectQueryBlock mysqlSelectQuery, Map, Map>> 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 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; } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidSingleUnitSelectParser.java b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidSingleUnitSelectParser.java index 2687483fc..d03afe712 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidSingleUnitSelectParser.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidSingleUnitSelectParser.java @@ -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 schemaMap = null; + private Map, 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 getSchemaMap() { + public Map, SchemaConfig> getSchemaMap() { return schemaMap; } - public void setSchemaMap(Map schemaMap) { + public void setSchemaMap(Map, SchemaConfig> schemaMap) { this.schemaMap = schemaMap; } } diff --git a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidUpdateParser.java b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidUpdateParser.java index e37b5bddc..44abae969 100644 --- a/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidUpdateParser.java +++ b/src/main/java/com/actiontech/dble/route/parser/druid/impl/DruidUpdateParser.java @@ -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); } } } diff --git a/src/main/java/com/actiontech/dble/route/util/ConditionUtil.java b/src/main/java/com/actiontech/dble/route/util/ConditionUtil.java new file mode 100644 index 000000000..62971666a --- /dev/null +++ b/src/main/java/com/actiontech/dble/route/util/ConditionUtil.java @@ -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 whereUnits, Map tableAliasMap, String defaultSchema) { + Iterator 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 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 outConditions = whereUnit.getOutAndConditions(); //outConditions item operator with AND + ListIterator 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 tableAliasMap, String defaultSchema, ListIterator iteratorConditions) { + while (iteratorConditions.hasNext()) { + TableStat.Condition condition = iteratorConditions.next(); + List 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 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 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 getTableInfo(Map 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 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> mergedConditions(List 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> mergedConditionList = getMergedConditionList(whereUnit.getSubWhereUnit()); + if (whereUnit.getOutAndConditions().size() > 0) { + for (List mergedCondition : mergedConditionList) { + mergedCondition.addAll(whereUnit.getOutAndConditions()); + } + } + whereUnit.getOrConditionList().addAll(mergedConditionList); + } else if (whereUnit.getOutAndConditions().size() > 0) { + whereUnit.getOrConditionList().add(whereUnit.getOutAndConditions()); + } + } + + private static List> getMergedConditionList(List whereUnitList) { + List> 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> merge(List> list1, List> list2) { + if (list1.size() == 0) { + return list2; + } else if (list2.size() == 0) { + return list1; + } + + List> retList = new ArrayList<>(); + for (List aList1 : list1) { + for (List aList2 : list2) { + List tmp = new ArrayList<>(); + tmp.addAll(aList1); + tmp.addAll(aList2); + retList.add(tmp); + } + } + return retList; + } + + public static void extendConditionsFromRelations(List conds, Set relations) { + List newConds = new ArrayList<>(); + Iterator 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 transformConditionToRouteUnits(List> conditionList) { + List retList = new ArrayList<>(); + //find partition column in condition + for (List aConditionList : conditionList) { + RouteCalculateUnit routeCalculateUnit = new RouteCalculateUnit(); + for (TableStat.Condition condition : aConditionList) { + List values = condition.getValues(); + String columnName = condition.getColumn().getName(); + String tableFullName = condition.getColumn().getTable(); + String operator = condition.getOperator(); + String[] tableInfo = tableFullName.split("\\."); + Pair 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 values) { + for (Object value : values) { + if (value != null && !value.toString().equals("")) { + return true; + } + } + return false; + } + + + public static List buildRouteCalculateUnits(List whereUnits, Map tableAliasMap, String defaultSchema) { + ConditionUtil.pruningConditions(whereUnits, tableAliasMap, defaultSchema); + if (whereUnits.size() == 0) { + WhereUnit whereUnit = new WhereUnit(); + whereUnit.setFinishedParse(true); + List> retList = new ArrayList<>(); + retList.add(new ArrayList<>()); + whereUnit.setOrConditionList(retList); + whereUnits.add(whereUnit); + } + List> conditions = ConditionUtil.mergedConditions(whereUnits); + return ConditionUtil.transformConditionToRouteUnits(conditions); + } +} diff --git a/src/main/java/com/actiontech/dble/route/util/RouterUtil.java b/src/main/java/com/actiontech/dble/route/util/RouterUtil.java index 9a00f6a03..4c7472002 100644 --- a/src/main/java/com/actiontech/dble/route/util/RouterUtil.java +++ b/src/main/java/com/actiontech/dble/route/util/RouterUtil.java @@ -21,6 +21,7 @@ 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.server.ServerConnection; import com.actiontech.dble.server.parser.ServerParse; import com.actiontech.dble.server.util.SchemaUtil; @@ -62,7 +63,7 @@ public final class RouterUtil { /** * removeSchema from sql * - * @param stmt + * @param stmt sql * @param schema has change to lowercase if need * @param isLowerCase lowercase * @return new sql @@ -87,7 +88,9 @@ public final class RouterUtil { flag = true; } else if (index1 >= 0 && index2 < 0) { flag = false; - } else flag = index2 < index1; + } else { + flag = index2 < index1; + } if (flag) { result.append(stmt.substring(startPos, index2)); startPos = index2 + maySchema2.length(); @@ -121,10 +124,11 @@ public final class RouterUtil { } - public static RouteResultset routeFromParserComplex(DruidParser druidParser, Map schemaMap, RouteResultset rrs, SQLStatement statement, - String originSql, LayerCachePool cachePool, ServerSchemaStatVisitor visitor, - ServerConnection sc, PlanNode node) throws SQLException { - druidParser.parser(null, rrs, statement, originSql, cachePool, visitor, sc); + public static RouteResultset routeFromParserComplex( + SchemaConfig schema, DruidParser druidParser, Map, SchemaConfig> schemaMap, + RouteResultset rrs, SQLStatement statement, String originSql, LayerCachePool cachePool, + ServerSchemaStatVisitor visitor, ServerConnection sc, PlanNode node) throws SQLException { + druidParser.parser(schema, rrs, statement, originSql, cachePool, visitor, sc); if (rrs.isFinishedExecute()) { return null; } @@ -140,8 +144,8 @@ public final class RouterUtil { SortedSet nodeSet = new TreeSet<>(); for (RouteCalculateUnit unit : druidParser.getCtx().getRouteCalculateUnits()) { - RouteResultset rrsTmp = RouterUtil.tryRouteForTablesComplex(schemaMap, druidParser.getCtx(), unit, rrs, isSelect(statement), cachePool, node); - if (rrsTmp != null && rrsTmp.getNodes() != null) { + RouteResultset rrsTmp = RouterUtil.tryRouteForTablesComplex(schemaMap, druidParser.getCtx(), unit, rrs, cachePool, node); + if (rrsTmp != null && (rrsTmp.getNodes() != null || rrsTmp.getNodes().length != 0)) { Collections.addAll(nodeSet, rrsTmp.getNodes()); if (rrsTmp.isGlobalTable()) { break; @@ -174,7 +178,7 @@ public final class RouterUtil { return rrs; } - /** + /* * no name table or others */ DruidShardingParseInfo ctx = druidParser.getCtx(); @@ -195,7 +199,7 @@ public final class RouterUtil { SortedSet nodeSet = new TreeSet<>(); for (RouteCalculateUnit unit : druidParser.getCtx().getRouteCalculateUnits()) { RouteResultset rrsTmp = RouterUtil.tryRouteForTables(schema, druidParser.getCtx(), unit, rrs, isSelect(statement), cachePool, node); - if (rrsTmp != null && rrsTmp.getNodes() != null) { + if (rrsTmp != null && (rrsTmp.getNodes() != null || rrsTmp.getNodes().length != 0)) { Collections.addAll(nodeSet, rrsTmp.getNodes()); if (rrsTmp.isGlobalTable()) { break; @@ -245,9 +249,9 @@ public final class RouterUtil { /** * the first node as the result * - * @param rrs - * @param dataNode - * @return + * @param rrs RouteResultset + * @param dataNode NAME + * @return RouteResultset */ public static RouteResultset routeToSingleNode(RouteResultset rrs, String dataNode) { if (dataNode == null) { @@ -305,53 +309,8 @@ public final class RouterUtil { return stmt.replaceAll("[\\t\\n\\r]", " ").trim(); } - /** - * getTableName - * - * @param stmt - * @param repPos - * @return name - * @author AStoneGod - */ - public static String getTableName(String stmt, int[] repPos) { - int startPos = repPos[0]; - int secInd = stmt.indexOf(' ', startPos + 1); - if (secInd < 0) { - secInd = stmt.length(); - } - int thiInd = stmt.indexOf('(', secInd + 1); - if (thiInd < 0) { - thiInd = stmt.length(); - } - repPos[1] = secInd; - String tableName = ""; - if (stmt.toUpperCase().startsWith("DESC") || stmt.toUpperCase().startsWith("DESCRIBE")) { - tableName = stmt.substring(startPos, thiInd).trim(); - } else { - tableName = stmt.substring(secInd, thiInd).trim(); - } - //ALTER TABLE - if (tableName.contains(" ")) { - tableName = tableName.substring(0, tableName.indexOf(" ")); - } - int ind2 = tableName.indexOf('.'); - if (ind2 > 0) { - tableName = tableName.substring(ind2 + 1); - } - return lowerCaseTable(tableName); - } - - - public static String lowerCaseTable(String tableName) { - if (DbleServer.getInstance().getSystemVariables().isLowerCaseTableNames()) { - return tableName.toLowerCase(); - } - return tableName; - } - - - public static RouteResultset routeToMultiNode(boolean cache, RouteResultset rrs, Collection dataNodes) { + private static RouteResultset routeToMultiNode(boolean cache, RouteResultset rrs, Collection dataNodes) { RouteResultsetNode[] nodes = new RouteResultsetNode[dataNodes.size()]; int i = 0; RouteResultsetNode node; @@ -365,7 +324,7 @@ public final class RouterUtil { } nodes[i++] = node; } - rrs.setCacheAble(cache); + rrs.setSqlRouteCacheAble(cache); rrs.setNodes(nodes); return rrs; } @@ -396,7 +355,7 @@ public final class RouterUtil { } } x.remove(index); - index = Math.abs(rand.nextInt(Integer.MAX_VALUE)) % x.size(); + index = rand.nextInt(x.size()); } return x.get(0); @@ -405,27 +364,24 @@ public final class RouterUtil { /** * getRandomDataNode * - * @param schema - * @param table + * @param schema SchemaConfig + * @param table NAME * @return datanode * @author mycat */ - private static String getRandomDataNode(SchemaConfig schema, - String table) throws SQLException { - String dataNode = null; + private static String getRandomDataNode(SchemaConfig schema, String table) throws SQLException { Map tables = schema.getTables(); TableConfig tc; if (tables != null && (tc = tables.get(table)) != null) { - dataNode = tc.getRandomDataNode(); + return tc.getRandomDataNode(); } else { String msg = "Table '" + schema.getName() + "." + table + "' doesn't exist"; throw new SQLException(msg, "42S02", ErrorCode.ER_NO_SUCH_TABLE); } - return dataNode; } - public static Set ruleByJoinValueCalculate(String sql, TableConfig tc, - Set colRoutePairSet) throws SQLNonTransientException { + private static Set ruleByJoinValueCalculate(String sql, TableConfig tc, + Set colRoutePairSet) throws SQLNonTransientException { Set retNodeSet = new LinkedHashSet<>(); if (tc.getDirectRouteTC() != null) { Set nodeSet = ruleCalculate(tc.getDirectRouteTC(), colRoutePairSet); @@ -461,7 +417,7 @@ public final class RouterUtil { } else if (colPair.rangeValue != null) { Integer[] nodeRange = algorithm.calculateRange(String.valueOf(colPair.rangeValue.getBeginValue()), String.valueOf(colPair.rangeValue.getEndValue())); if (nodeRange != null) { - /** + /* * not sure colPair's nodeid has other effect */ if (nodeRange.length == 0) { @@ -481,95 +437,131 @@ public final class RouterUtil { return routeNodeSet; } - public static String tryRouteTablesToOneNode(String user, RouteResultset rrs, SchemaConfig schemaConfig, DruidShardingParseInfo ctx, Set schemaList, int tableSize, boolean isSelect) throws SQLException { + public static String tryRouteTablesToOneNode(RouteResultset rrs, DruidShardingParseInfo ctx, Set schemaList, int tableSize, boolean isSelect) throws SQLException { if (ctx.getTables().size() != tableSize) { return null; } - Set resultNodes = new HashSet<>(); - for (RouteCalculateUnit routeUnit : ctx.getRouteCalculateUnits()) { - Set tmpResultNodes = new HashSet<>(); - Set globalTables = new HashSet<>(); - Set tablesSet = new HashSet<>(ctx.getTables()); - Map>> tablesAndConditions = routeUnit.getTablesAndConditions(); - if (tablesAndConditions != null) { - for (Map.Entry>> entry : tablesAndConditions.entrySet()) { - String fullTable = entry.getKey(); - SchemaInfo schemaInfo = SchemaUtil.getSchemaInfo(user, schemaConfig, fullTable); - SchemaConfig schema = schemaInfo.getSchemaConfig(); - String schemaName = schemaInfo.getSchema(); - String tableName = schemaInfo.getTable(); - schemaList.add(schemaName); - TableConfig tableConfig = schema.getTables().get(tableName); - if (tableConfig == null) { - if (schema.getDataNode() == null) { - String msg = " Table '" + schemaName + "." + tableName + "' doesn't exist"; - LOGGER.info(msg); - throw new SQLNonTransientException(msg); - } else { - tmpResultNodes.add(schema.getDataNode()); - tablesSet.remove(tableName); - if (tmpResultNodes.size() != 1) { - return null; - } - } - } else if (tableConfig.isGlobalTable()) { - globalTables.add(tableConfig); - } else if (schema.getTables().get(tableName).getDataNodes().size() == 1) { - tmpResultNodes.add(schema.getTables().get(tableName).getDataNodes().get(0)); - tablesSet.remove(tableName); - if (tmpResultNodes.size() != 1) { - return null; - } - } else { - if (!tryCalcNodeForShardingTable(rrs, tmpResultNodes, tablesSet, entry, tableName, schema, tableConfig, isSelect)) { - return null; - } - } - } - } - if (globalTables.size() == tableSize) { - boolean isFirstTable = true; - for (TableConfig tb : globalTables) { - if (isFirstTable) { - tmpResultNodes.addAll(tb.getDataNodes()); - isFirstTable = false; - } else { - tmpResultNodes.retainAll(tb.getDataNodes()); - } - } - if (tmpResultNodes.size() != 0) { - return getRandomDataNode(new ArrayList<>(tmpResultNodes)); - } else { + Set tmpResultNodes = new HashSet<>(); + + Set> tablesSet = new HashSet<>(ctx.getTables()); + Set> globalTables = new HashSet<>(); + for (Pair table : ctx.getTables()) { + String schemaName = table.getKey(); + String tableName = table.getValue(); + SchemaConfig schema = DbleServer.getInstance().getConfig().getSchemas().get(schemaName); + schemaList.add(schemaName); + TableConfig tableConfig = schema.getTables().get(tableName); + if (tableConfig == null) { + if (tryRouteNoShardingTablesToOneNode(tmpResultNodes, tablesSet, table, schemaName, tableName, schema)) + return null; + } else if (tableConfig.isGlobalTable()) { + globalTables.add(new Pair<>(schemaName, tableConfig)); + } else if (schema.getTables().get(tableName).getDataNodes().size() == 1) { + tmpResultNodes.add(schema.getTables().get(tableName).getDataNodes().get(0)); + tablesSet.remove(table); + if (tmpResultNodes.size() != 1) { return null; } } - for (TableConfig tb : globalTables) { - tmpResultNodes.retainAll(tb.getDataNodes()); - tablesSet.remove(tb.getName()); + } + if (globalTables.size() == tableSize) { + return tryRouteGlobalTablesToOneNode(tmpResultNodes, globalTables); + } + if (tablesSet.size() != 0) { + Set resultNodes = new HashSet<>(); + for (RouteCalculateUnit routeUnit : ctx.getRouteCalculateUnits()) { + Map, Map>> tablesAndConditions = routeUnit.getTablesAndConditions(); + if (tablesAndConditions != null) { + for (Map.Entry, Map>> entry : tablesAndConditions.entrySet()) { + Pair table = entry.getKey(); + String schemaName = table.getKey(); + String tableName = table.getValue(); + SchemaConfig schema = DbleServer.getInstance().getConfig().getSchemas().get(schemaName); + TableConfig tableConfig = schema.getTables().get(tableName); + if (!tryCalcNodeForShardingColumn(rrs, tmpResultNodes, tablesSet, entry, table, tableConfig, isSelect)) { + return null; + } + } + } + for (Pair table : globalTables) { + TableConfig tb = table.getValue(); + tmpResultNodes.retainAll(tb.getDataNodes()); + tablesSet.remove(new Pair<>(table.getKey(), tb.getName())); + } + if (tmpResultNodes.size() != 1 || tablesSet.size() != 0) { + return null; + } + resultNodes.add(tmpResultNodes.iterator().next()); + if (resultNodes.size() != 1) { + return null; + } } - if (tmpResultNodes.size() != 1 || tablesSet.size() != 0) { + if (resultNodes.size() != 1) { return null; } - resultNodes.add(tmpResultNodes.iterator().next()); + return resultNodes.iterator().next(); + } else { + if (tmpResultNodes.size() != 1) { + return null; + } + return tmpResultNodes.iterator().next(); } - if (resultNodes.size() != 1) { - return null; - } - return resultNodes.iterator().next(); } - private static boolean tryCalcNodeForShardingTable(RouteResultset rrs, Set resultNodes, Set tablesSet, Map.Entry>> entry, String tableName, SchemaConfig schema, TableConfig tableConfig, boolean isSelect) throws SQLNonTransientException { + private static String tryRouteGlobalTablesToOneNode(Set tmpResultNodes, Set> globalTables) { + boolean isFirstTable = true; + for (Pair table : globalTables) { + TableConfig tb = table.getValue(); + if (isFirstTable) { + tmpResultNodes.addAll(tb.getDataNodes()); + isFirstTable = false; + } else { + tmpResultNodes.retainAll(tb.getDataNodes()); + } + } + if (tmpResultNodes.size() != 0) { + return getRandomDataNode(new ArrayList<>(tmpResultNodes)); + } else { + return null; + } + } + + private static boolean tryRouteNoShardingTablesToOneNode(Set tmpResultNodes, Set> tablesSet, Pair table, String schemaName, String tableName, SchemaConfig schema) throws SQLNonTransientException { + if (schema.getDataNode() == null) { + String msg = " Table '" + schemaName + "." + tableName + "' doesn't exist"; + LOGGER.info(msg); + throw new SQLNonTransientException(msg); + } else { + tmpResultNodes.add(schema.getDataNode()); + tablesSet.remove(table); + if (tmpResultNodes.size() != 1) { + return true; + } + } + return false; + } + + private static boolean tryCalcNodeForShardingColumn( + RouteResultset rrs, Set resultNodes, Set> tablesSet, + Map.Entry, Map>> entry, Pair table, + TableConfig tableConfig, boolean isSelect) throws SQLNonTransientException { + if (tableConfig == null) { + return false; // alias table, may subquery + } + if (tableConfig.getPartitionColumn() == null) { + return true; + } Map> columnsMap = entry.getValue(); - Map> tablesRouteMap = new HashMap<>(); - if (tryRouteWithPrimaryCache(rrs, tablesRouteMap, DbleServer.getInstance().getRouterService().getTableId2DataNodeCache(), columnsMap, schema, tableName, tableConfig.getPrimaryKey(), isSelect)) { - Set nodes = tablesRouteMap.get(tableName); + Map, Set> tablesRouteMap = new HashMap<>(); + if (tryRouteWithCache(rrs, tablesRouteMap, DbleServer.getInstance().getRouterService().getTableId2DataNodeCache(), columnsMap, table, tableConfig.getPrimaryKey(), isSelect)) { + Set nodes = tablesRouteMap.get(table); if (nodes == null || nodes.size() != 1) { return false; } else { resultNodes.add(nodes.iterator().next()); - tablesSet.remove(tableName); + tablesSet.remove(table); return true; } } @@ -579,47 +571,7 @@ public final class RouterUtil { // where filter contains partition column if (partitionCol != null && columnsMap.get(partitionCol) != null) { - Set partitionValue = columnsMap.get(partitionCol); - if (partitionValue.size() == 0) { - return false; - } else { - for (ColumnRoutePair pair : partitionValue) { - AbstractPartitionAlgorithm algorithm = tableConfig.getRule().getRuleAlgorithm(); - if (pair.colValue != null && !"null".equals(pair.colValue)) { - Integer nodeIndex; - try { - nodeIndex = algorithm.calculate(pair.colValue); - } catch (Exception e) { - return false; - } - if (nodeIndex == null) { - String msg = "can't find any valid data node :" + tableConfig.getName() + - " -> " + tableConfig.getPartitionColumn() + " -> " + pair.colValue; - LOGGER.info(msg); - throw new SQLNonTransientException(msg); - } - - ArrayList dataNodes = tableConfig.getDataNodes(); - String node; - if (nodeIndex >= 0 && nodeIndex < dataNodes.size()) { - node = dataNodes.get(nodeIndex); - } else { - String msg = "Can't find a valid data node for specified node index :" + - tableConfig.getName() + " -> " + tableConfig.getPartitionColumn() + - " -> " + pair.colValue + " -> " + "Index : " + nodeIndex; - LOGGER.info(msg); - throw new SQLNonTransientException(msg); - } - if (node != null) { - resultNodes.add(node); - tablesSet.remove(tableName); - if (resultNodes.size() != 1) { - return false; - } - } - } - } - } + if (tryCalcNodeForShardingColumn(resultNodes, tablesSet, table, tableConfig, columnsMap, partitionCol)) return false; } else if (joinKey != null && columnsMap.get(joinKey) != null && columnsMap.get(joinKey).size() != 0) { Set joinKeyValue = columnsMap.get(joinKey); Set dataNodeSet = ruleByJoinValueCalculate(rrs.getStatement(), tableConfig, joinKeyValue); @@ -627,42 +579,85 @@ public final class RouterUtil { return false; } resultNodes.addAll(dataNodeSet); - tablesSet.remove(tableName); - if (resultNodes.size() != 1) { - return false; - } + tablesSet.remove(table); + return resultNodes.size() == 1; } else { return false; } return true; } + private static boolean tryCalcNodeForShardingColumn(Set resultNodes, Set> tablesSet, Pair table, TableConfig tableConfig, Map> columnsMap, String partitionCol) throws SQLNonTransientException { + Set partitionValue = columnsMap.get(partitionCol); + if (partitionValue.size() == 0) { + return true; + } else { + for (ColumnRoutePair pair : partitionValue) { + AbstractPartitionAlgorithm algorithm = tableConfig.getRule().getRuleAlgorithm(); + if (pair.colValue != null && !"null".equals(pair.colValue)) { + Integer nodeIndex; + try { + nodeIndex = algorithm.calculate(pair.colValue); + } catch (Exception e) { + return true; + } + if (nodeIndex == null) { + String msg = "can't find any valid data node :" + tableConfig.getName() + + " -> " + tableConfig.getPartitionColumn() + " -> " + pair.colValue; + LOGGER.info(msg); + throw new SQLNonTransientException(msg); + } + + ArrayList dataNodes = tableConfig.getDataNodes(); + String node; + if (nodeIndex >= 0 && nodeIndex < dataNodes.size()) { + node = dataNodes.get(nodeIndex); + } else { + String msg = "Can't find a valid data node for specified node index :" + + tableConfig.getName() + " -> " + tableConfig.getPartitionColumn() + + " -> " + pair.colValue + " -> " + "Index : " + nodeIndex; + LOGGER.info(msg); + throw new SQLNonTransientException(msg); + } + if (node != null) { + resultNodes.add(node); + tablesSet.remove(table); + if (resultNodes.size() != 1) { + return true; + } + } + } + } + } + return false; + } + /** * tryRouteFor multiTables + * */ - public static RouteResultset tryRouteForTables(SchemaConfig schema, DruidShardingParseInfo ctx, - RouteCalculateUnit routeUnit, RouteResultset rrs, boolean isSelect, LayerCachePool cachePool, PlanNode node) - throws SQLException { - - List tables = ctx.getTables(); + private static RouteResultset tryRouteForTables( + SchemaConfig schema, DruidShardingParseInfo ctx, RouteCalculateUnit routeUnit, RouteResultset rrs, + boolean isSelect, LayerCachePool cachePool, PlanNode node) throws SQLException { + List> tables = ctx.getTables(); // no sharding table - String noShardingNode = RouterUtil.isNoSharding(schema, tables.get(0)); + String noShardingNode = RouterUtil.isNoSharding(schema, tables.get(0).getValue()); if (noShardingNode != null) { return RouterUtil.routeToSingleNode(rrs, noShardingNode); } if (tables.size() == 1) { - return RouterUtil.tryRouteForOneTable(schema, routeUnit, tables.get(0), rrs, isSelect, cachePool, node); + return RouterUtil.tryRouteForOneTable(schema, routeUnit, tables.get(0).getValue(), rrs, isSelect, cachePool, node); } - /** + /* * multi-table it must be ER OR global* normal , global* er */ //map - Map> tablesRouteMap = new HashMap<>(); + Map, Set> tablesRouteMap = new HashMap<>(); - Map>> tablesAndConditions = routeUnit.getTablesAndConditions(); + Map, Map>> tablesAndConditions = routeUnit.getTablesAndConditions(); if (tablesAndConditions != null && tablesAndConditions.size() > 0) { //findRouter for shard-ing table RouterUtil.findRouterWithConditionsForTables(schema, rrs, tablesAndConditions, tablesRouteMap, cachePool, isSelect, false, node); @@ -673,39 +668,20 @@ public final class RouterUtil { //findRouter for singe table global table will not change the result // if global table and normal table has no intersection ,they had treat as normal join - for (String tableName : tables) { + for (Pair table : tables) { + String tableName = table.getValue(); String testShardingNode = RouterUtil.isNoSharding(schema, tableName); if (testShardingNode != null && tablesRouteMap.size() == 0) { return RouterUtil.routeToSingleNode(rrs, testShardingNode); } TableConfig tableConfig = schema.getTables().get(tableName); - if (tableConfig != null && !tableConfig.isGlobalTable() && tablesRouteMap.get(tableName) == null) { //the other is single table - tablesRouteMap.put(tableName, new HashSet()); - tablesRouteMap.get(tableName).addAll(tableConfig.getDataNodes()); + if (tableConfig != null && !tableConfig.isGlobalTable() && tablesRouteMap.get(table) == null) { //the other is single table + tablesRouteMap.put(table, new HashSet<>()); + tablesRouteMap.get(table).addAll(tableConfig.getDataNodes()); } } - Set retNodesSet = new HashSet<>(); - boolean isFirstAdd = true; - for (Map.Entry> entry : tablesRouteMap.entrySet()) { - if (entry.getValue() == null || entry.getValue().size() == 0) { - throw new SQLNonTransientException("parent key can't find any valid datanode "); - } else { - if (isFirstAdd) { - retNodesSet.addAll(entry.getValue()); - isFirstAdd = false; - } else { - retNodesSet.retainAll(entry.getValue()); - if (retNodesSet.size() == 0) { //two tables has no no intersection - String errMsg = "invalid route in sql, multi tables found but datanode has no intersection " + - " sql:" + rrs.getStatement(); - LOGGER.info(errMsg); - throw new SQLNonTransientException(errMsg); - } - } - } - } - //retNodesSet.size() >0 + Set retNodesSet = retainRouteMap(tablesRouteMap); routeToMultiNode(isSelect, rrs, retNodesSet); return rrs; @@ -715,32 +691,33 @@ public final class RouterUtil { /** * tryRouteFor multiTables */ - public static RouteResultset tryRouteForTablesComplex(Map schemaMap, DruidShardingParseInfo ctx, - RouteCalculateUnit routeUnit, RouteResultset rrs, boolean isSelect, LayerCachePool cachePool, PlanNode node) + private static RouteResultset tryRouteForTablesComplex(Map, SchemaConfig> schemaMap, DruidShardingParseInfo ctx, + RouteCalculateUnit routeUnit, RouteResultset rrs, LayerCachePool cachePool, PlanNode node) throws SQLException { - List tables = ctx.getTables(); + List> tables = ctx.getTables(); + Pair firstTable = tables.get(0); // no sharding table - String noShardingNode = RouterUtil.isNoSharding(schemaMap.get(tables.get(0)), tables.get(0)); + String noShardingNode = RouterUtil.isNoSharding(schemaMap.get(firstTable), firstTable.getValue()); if (noShardingNode != null) { return RouterUtil.routeToSingleNode(rrs, noShardingNode); } if (tables.size() == 1) { - return RouterUtil.tryRouteForOneTable((schemaMap.get(tables.get(0))), routeUnit, tables.get(0), rrs, isSelect, cachePool, node); + return RouterUtil.tryRouteForOneTable(schemaMap.get(firstTable), routeUnit, firstTable.getValue(), rrs, true, cachePool, node); } - /** + /* * multi-table it must be ER OR global* normal , global* er */ //map - Map> tablesRouteMap = new HashMap<>(); + Map, Set> tablesRouteMap = new HashMap<>(); - Map>> tablesAndConditions = routeUnit.getTablesAndConditions(); + Map, Map>> tablesAndConditions = routeUnit.getTablesAndConditions(); if (tablesAndConditions != null && tablesAndConditions.size() > 0) { //findRouter for shard-ing table - RouterUtil.findRouterWithConditionsForTables(schemaMap, rrs, tablesAndConditions, tablesRouteMap, cachePool, isSelect, false, node); + RouterUtil.findRouterWithConditionsForTables(schemaMap, rrs, tablesAndConditions, tablesRouteMap, cachePool, node); if (rrs.isFinishedRoute()) { return rrs; } @@ -748,22 +725,29 @@ public final class RouterUtil { //findRouter for singe table global table will not change the result // if global table and normal table has no intersection ,they had treat as normal join - for (String tableName : tables) { - SchemaConfig schema = schemaMap.get(tableName); + for (Pair table : tables) { + SchemaConfig schema = DbleServer.getInstance().getConfig().getSchemas().get(table.getKey()); + String tableName = table.getValue(); String testShardingNode = RouterUtil.isNoSharding(schema, tableName); if (testShardingNode != null && tablesRouteMap.size() == 0) { return RouterUtil.routeToSingleNode(rrs, testShardingNode); } TableConfig tableConfig = schema.getTables().get(tableName); - if (tableConfig != null && !tableConfig.isGlobalTable() && tablesRouteMap.get(tableName) == null) { //the other is single table - tablesRouteMap.put(tableName, new HashSet()); - tablesRouteMap.get(tableName).addAll(tableConfig.getDataNodes()); + if (tableConfig != null && !tableConfig.isGlobalTable() && tablesRouteMap.get(table) == null) { //the other is single table + tablesRouteMap.put(table, new HashSet<>()); + tablesRouteMap.get(table).addAll(tableConfig.getDataNodes()); } } + Set retNodesSet = retainRouteMap(tablesRouteMap); + routeToMultiNode(true, rrs, retNodesSet); + return rrs; + } + + private static Set retainRouteMap(Map, Set> tablesRouteMap) throws SQLNonTransientException { Set retNodesSet = new HashSet<>(); boolean isFirstAdd = true; - for (Map.Entry> entry : tablesRouteMap.entrySet()) { + for (Map.Entry, Set> entry : tablesRouteMap.entrySet()) { if (entry.getValue() == null || entry.getValue().size() == 0) { throw new SQLNonTransientException("parent key can't find any valid datanode "); } else { @@ -772,19 +756,13 @@ public final class RouterUtil { isFirstAdd = false; } else { retNodesSet.retainAll(entry.getValue()); - if (retNodesSet.size() == 0) { //two tables has no no intersection - String errMsg = "invalid route in sql, multi tables found but datanode has no intersection " + - " sql:" + rrs.getStatement(); - LOGGER.info(errMsg); - throw new SQLNonTransientException(errMsg); + if (retNodesSet.size() == 0) { + return retNodesSet; } } } } - //retNodesSet.size() >0 - routeToMultiNode(isSelect, rrs, retNodesSet); - return rrs; - + return retNodesSet; } @@ -804,7 +782,7 @@ public final class RouterUtil { if (tc.isGlobalTable()) { if (isSelect) { // global select ,not cache route result - rrs.setCacheAble(false); + rrs.setSqlRouteCacheAble(false); rrs.setGlobalTable(true); String randomDataNode = tc.getDataNodes().get(0); //tc.getRandomDataNode(); rrs = routeToSingleNode(rrs, randomDataNode); @@ -820,37 +798,64 @@ public final class RouterUtil { return routeToMultiNode(false, rrs, tc.getDataNodes(), true); } } else { //single table or shard-ing table - if (!checkRuleRequired(schema, routeUnit, tc)) { - throw new IllegalArgumentException("route rule for table " + + + Pair table = new Pair<>(schema.getName(), tableName); + if (!checkRuleRequired(schema, routeUnit, tc, table)) { + throw new IllegalArgumentException("route rule for table " + schema.getName() + "." + tc.getName() + " is required: " + rrs.getStatement()); } if ((tc.getPartitionColumn() == null && tc.getParentTC() == null) || (tc.getParentTC() != null && tc.getDirectRouteTC() == null)) { // single table or one of the children of complex ER table - return routeToMultiNode(rrs.isCacheAble(), rrs, tc.getDataNodes()); + return routeToMultiNode(rrs.isSqlRouteCacheAble(), rrs, tc.getDataNodes()); } else { - Map> tablesRouteMap = new HashMap<>(); + Map, Set> tablesRouteMap = new HashMap<>(); if (routeUnit.getTablesAndConditions() != null && routeUnit.getTablesAndConditions().size() > 0) { RouterUtil.findRouterWithConditionsForTables(schema, rrs, routeUnit.getTablesAndConditions(), tablesRouteMap, cachePool, isSelect, true, node); if (rrs.isFinishedRoute()) { return rrs; } } - if (tablesRouteMap.get(tableName) == null) { - return routeToMultiNode(rrs.isCacheAble(), rrs, tc.getDataNodes()); + if (tablesRouteMap.get(table) == null) { + return routeToMultiNode(rrs.isSqlRouteCacheAble(), rrs, tc.getDataNodes()); } else { - return routeToMultiNode(rrs.isCacheAble(), rrs, tablesRouteMap.get(tableName)); + return routeToMultiNode(rrs.isSqlRouteCacheAble(), rrs, tablesRouteMap.get(table)); } } } } + /** + * @param schema SchemaConfig + * @param tc TableConfig + * @return true for passed + */ + private static boolean checkRuleRequired(SchemaConfig schema, RouteCalculateUnit routeUnit, TableConfig tc, Pair table) { + if (!tc.isRuleRequired()) { + return true; + } + boolean hasRequiredValue = false; + if (routeUnit.getTablesAndConditions().get(table) == null || routeUnit.getTablesAndConditions().get(table).size() == 0) { + hasRequiredValue = false; + } else { + for (Map.Entry> condition : routeUnit.getTablesAndConditions().get(table).entrySet()) { + + String colName = RouterUtil.getFixedSql(RouterUtil.removeSchema(condition.getKey(), schema.getName())); + //condition is partition column + if (colName.equals(tc.getPartitionColumn())) { + hasRequiredValue = true; + break; + } + } + } + return hasRequiredValue; + } - private static boolean tryRouteWithPrimaryCache( - RouteResultset rrs, Map> tablesRouteMap, + private static boolean tryRouteWithCache( + RouteResultset rrs, Map, Set> tablesRouteMap, LayerCachePool cachePool, Map> columnsMap, - SchemaConfig schema, String tableName, String primaryKey, boolean isSelect) { + Pair table, String primaryKey, boolean isSelect) { if (cachePool == null || primaryKey == null || columnsMap.get(primaryKey) == null) { return false; } @@ -864,7 +869,7 @@ public final class RouterUtil { LOGGER.debug("try to find cache by primary key "); } - String tableKey = StringUtil.getFullName(schema.getName(), tableName, '_'); + String tableKey = StringUtil.getFullName(table.getKey(), table.getValue(), '_'); boolean allFound = true; for (ColumnRoutePair pair : primaryKeyPairs) { // may be has multi value of primary key, eg: in(1,2,3) String cacheKey = pair.colValue; @@ -873,10 +878,8 @@ public final class RouterUtil { allFound = false; break; } else { - if (tablesRouteMap.get(tableName) == null) { - tablesRouteMap.put(tableName, new HashSet()); - } - tablesRouteMap.get(tableName).add(dataNode); + tablesRouteMap.computeIfAbsent(table, k -> new HashSet<>()); + tablesRouteMap.get(table).add(dataNode); } } if (!allFound && isSelect) { @@ -890,66 +893,26 @@ public final class RouterUtil { /** * findRouterWithConditionsForTables */ - public static void findRouterWithConditionsForTables(Map schemaMap, RouteResultset rrs, - Map>> tablesAndConditions, - Map> tablesRouteMap, LayerCachePool cachePool, - boolean isSelect, boolean isSingleTable, PlanNode node) throws SQLNonTransientException { + private static void findRouterWithConditionsForTables( + Map, SchemaConfig> schemaMap, RouteResultset rrs, + Map, Map>> tablesAndConditions, + Map, Set> tablesRouteMap, LayerCachePool cachePool, + PlanNode node) throws SQLNonTransientException { //router for shard-ing tables - for (Map.Entry>> entry : tablesAndConditions.entrySet()) { - String tableName = entry.getKey(); - SchemaConfig schema = schemaMap.get(tableName); - if (DbleServer.getInstance().getSystemVariables().isLowerCaseTableNames()) { - tableName = tableName.toLowerCase(); - } - if (tableName.startsWith(schema.getName() + ".")) { - tableName = tableName.substring(schema.getName().length() + 1); - } + for (Map.Entry, Map>> entry : tablesAndConditions.entrySet()) { + Pair table = entry.getKey(); + String tableName = table.getValue(); + SchemaConfig schema = schemaMap.get(table); TableConfig tableConfig = schema.getTables().get(tableName); - if (tableConfig == null) { - if (isSingleTable) { - String msg = " Table '" + schema.getName() + "." + tableName + "' doesn't exist"; - LOGGER.info(msg); - throw new SQLNonTransientException(msg); - } else { - //cross to other schema - continue; - } - } - if (tableConfig.isGlobalTable() || schema.getTables().get(tableName).getDataNodes().size() == 1) { - //global table or single node shard-ing table will router later - continue; - } else { //shard-ing table,childTable or others + if (tableConfig != null && !tableConfig.isGlobalTable() && schema.getTables().get(tableName).getDataNodes().size() != 1) { + //shard-ing table,childTable or others Map> columnsMap = entry.getValue(); - if (tryRouteWithPrimaryCache(rrs, tablesRouteMap, cachePool, columnsMap, schema, tableName, tableConfig.getPrimaryKey(), isSelect)) { + if (tryRouteWithCache(rrs, tablesRouteMap, cachePool, columnsMap, table, tableConfig.getPrimaryKey(), true)) { continue; } - String joinKey = tableConfig.getJoinKey(); - String partitionCol = tableConfig.getPartitionColumn(); - boolean isFoundPartitionValue = partitionCol != null && columnsMap.get(partitionCol) != null; - - // where filter contains partition column - if (isFoundPartitionValue) { - Set partitionValue = columnsMap.get(partitionCol); - if (partitionValue.size() == 0) { - if (tablesRouteMap.get(tableName) == null) { - tablesRouteMap.put(tableName, new HashSet()); - } - tablesRouteMap.get(tableName).addAll(tableConfig.getDataNodes()); - } else { - routeWithPartition(tablesRouteMap, tableName, tableConfig, partitionValue, node); - } - } else if (joinKey != null && columnsMap.get(joinKey) != null && columnsMap.get(joinKey).size() != 0) { - routerForJoinTable(rrs, tableConfig, columnsMap, joinKey); - return; - } else { - //no partition column,router to all nodes - if (tablesRouteMap.get(tableName) == null) { - tablesRouteMap.put(tableName, new HashSet()); - } - tablesRouteMap.get(tableName).addAll(tableConfig.getDataNodes()); - } + if (findRouterWithConditionsForTable(rrs, tablesRouteMap, node, table, tableConfig, columnsMap)) return; } } } @@ -958,20 +921,15 @@ public final class RouterUtil { /** * findRouterWithConditionsForTables */ - public static void findRouterWithConditionsForTables(SchemaConfig schema, RouteResultset rrs, - Map>> tablesAndConditions, - Map> tablesRouteMap, LayerCachePool cachePool, - boolean isSelect, boolean isSingleTable, PlanNode node) throws SQLNonTransientException { + private static void findRouterWithConditionsForTables( + SchemaConfig schema, RouteResultset rrs, Map, Map>> tablesAndConditions, + Map, Set> tablesRouteMap, LayerCachePool cachePool, + boolean isSelect, boolean isSingleTable, PlanNode node) throws SQLNonTransientException { //router for shard-ing tables - for (Map.Entry>> entry : tablesAndConditions.entrySet()) { - String tableName = entry.getKey(); - if (DbleServer.getInstance().getSystemVariables().isLowerCaseTableNames()) { - tableName = tableName.toLowerCase(); - } - if (tableName.startsWith(schema.getName() + ".")) { - tableName = tableName.substring(schema.getName().length() + 1); - } + for (Map.Entry, Map>> entry : tablesAndConditions.entrySet()) { + Pair table = entry.getKey(); + String tableName = table.getValue(); TableConfig tableConfig = schema.getTables().get(tableName); if (tableConfig == null) { if (isSingleTable) { @@ -983,45 +941,48 @@ public final class RouterUtil { continue; } } - if (tableConfig.isGlobalTable() || schema.getTables().get(tableName).getDataNodes().size() == 1) { - //global table or single node shard-ing table will router later - continue; - } else { //shard-ing table,childTable or others + //shard-ing table,childTable or others . global table or single node shard-ing table will router later + if (!tableConfig.isGlobalTable() && schema.getTables().get(tableName).getDataNodes().size() != 1) { + Map> columnsMap = entry.getValue(); - if (tryRouteWithPrimaryCache(rrs, tablesRouteMap, cachePool, columnsMap, schema, tableName, tableConfig.getPrimaryKey(), isSelect)) { + if (tryRouteWithCache(rrs, tablesRouteMap, cachePool, columnsMap, table, tableConfig.getPrimaryKey(), isSelect)) { continue; } - String joinKey = tableConfig.getJoinKey(); - String partitionCol = tableConfig.getPartitionColumn(); - boolean isFoundPartitionValue = partitionCol != null && columnsMap.get(partitionCol) != null; - - // where filter contains partition column - if (isFoundPartitionValue) { - Set partitionValue = columnsMap.get(partitionCol); - if (partitionValue.size() == 0) { - if (tablesRouteMap.get(tableName) == null) { - tablesRouteMap.put(tableName, new HashSet()); - } - tablesRouteMap.get(tableName).addAll(tableConfig.getDataNodes()); - } else { - routeWithPartition(tablesRouteMap, tableName, tableConfig, partitionValue, node); - } - } else if (joinKey != null && columnsMap.get(joinKey) != null && columnsMap.get(joinKey).size() != 0) { - routerForJoinTable(rrs, tableConfig, columnsMap, joinKey); - return; - } else { - //no partition column,router to all nodes - if (tablesRouteMap.get(tableName) == null) { - tablesRouteMap.put(tableName, new HashSet()); - } - tablesRouteMap.get(tableName).addAll(tableConfig.getDataNodes()); - } + if (findRouterWithConditionsForTable(rrs, tablesRouteMap, node, table, tableConfig, columnsMap)) return; } } } - private static void routeWithPartition(Map> tablesRouteMap, String tableName, TableConfig tableConfig, Set partitionValue, PlanNode pnode) throws SQLNonTransientException { + private static boolean findRouterWithConditionsForTable( + RouteResultset rrs, Map, Set> tablesRouteMap, + PlanNode node, Pair table, TableConfig tableConfig, + Map> columnsMap) throws SQLNonTransientException { + String joinKey = tableConfig.getJoinKey(); + String partitionCol = tableConfig.getPartitionColumn(); + boolean isFoundPartitionValue = partitionCol != null && columnsMap.get(partitionCol) != null; + + // where filter contains partition column + if (isFoundPartitionValue) { + Set partitionValue = columnsMap.get(partitionCol); + if (partitionValue.size() == 0) { + tablesRouteMap.computeIfAbsent(table, k -> new HashSet<>()); + tablesRouteMap.get(table).addAll(tableConfig.getDataNodes()); + } else { + routeWithPartition(tablesRouteMap, table, tableConfig, partitionValue, node); + } + } else if (joinKey != null && columnsMap.get(joinKey) != null && columnsMap.get(joinKey).size() != 0) { + routerForJoinTable(rrs, tableConfig, columnsMap, joinKey); + return true; + } else { + //no partition column,router to all nodes + tablesRouteMap.computeIfAbsent(table, k -> new HashSet<>()); + tablesRouteMap.get(table).addAll(tableConfig.getDataNodes()); + } + return false; + } + + private static void routeWithPartition(Map, Set> tablesRouteMap, Pair table, TableConfig tableConfig, Set partitionValue, PlanNode pnode) throws SQLNonTransientException { for (ColumnRoutePair pair : partitionValue) { AbstractPartitionAlgorithm algorithm = tableConfig.getRule().getRuleAlgorithm(); if (pnode != null && "null".equals(pair.colValue)) { @@ -1041,24 +1002,17 @@ public final class RouterUtil { } ArrayList dataNodes = tableConfig.getDataNodes(); - String node; if (nodeIndex >= 0 && nodeIndex < dataNodes.size()) { - node = dataNodes.get(nodeIndex); - + String node = dataNodes.get(nodeIndex); + tablesRouteMap.computeIfAbsent(table, k -> new HashSet<>()); + tablesRouteMap.get(table).add(node); } else { - node = null; String msg = "Can't find a valid data node for specified node index :" + tableConfig.getName() + " -> " + tableConfig.getPartitionColumn() + " -> " + pair.colValue + " -> " + "Index : " + nodeIndex; LOGGER.info(msg); throw new SQLNonTransientException(msg); } - if (node != null) { - if (tablesRouteMap.get(tableName) == null) { - tablesRouteMap.put(tableName, new HashSet()); - } - tablesRouteMap.get(tableName).add(node); - } } if (pair.rangeValue != null) { Integer[] nodeIndexes = algorithm.calculateRange( @@ -1075,10 +1029,8 @@ public final class RouterUtil { throw new SQLNonTransientException(msg); } if (node != null) { - if (tablesRouteMap.get(tableName) == null) { - tablesRouteMap.put(tableName, new HashSet()); - } - tablesRouteMap.get(tableName).add(node); + tablesRouteMap.computeIfAbsent(table, k -> new HashSet<>()); + tablesRouteMap.get(table).add(node); } } } @@ -1099,42 +1051,15 @@ public final class RouterUtil { Arrays.toString(dataNodeSet.toArray()) + " sql :" + rrs.getStatement()); } if (dataNodeSet.size() > 1) { - routeToMultiNode(rrs.isCacheAble(), rrs, dataNodeSet); + routeToMultiNode(rrs.isSqlRouteCacheAble(), rrs, dataNodeSet); rrs.setFinishedRoute(true); - return; } else { - rrs.setCacheAble(true); + rrs.setSqlRouteCacheAble(true); routeToSingleNode(rrs, dataNodeSet.iterator().next()); - return; } } - /** - * @param schema - * @param tc - * @return true for passed - */ - public static boolean checkRuleRequired(SchemaConfig schema, RouteCalculateUnit routeUnit, TableConfig tc) { - if (!tc.isRuleRequired()) { - return true; - } - boolean hasRequiredValue = false; - String tableName = tc.getName(); - if (routeUnit.getTablesAndConditions().get(tableName) == null || routeUnit.getTablesAndConditions().get(tableName).size() == 0) { - hasRequiredValue = false; - } else { - for (Map.Entry> condition : routeUnit.getTablesAndConditions().get(tableName).entrySet()) { - String colName = RouterUtil.getFixedSql(RouterUtil.removeSchema(condition.getKey(), schema.getName())); - //condition is partition column - if (colName.equals(tc.getPartitionColumn())) { - hasRequiredValue = true; - break; - } - } - } - return hasRequiredValue; - } /** @@ -1187,25 +1112,15 @@ public final class RouterUtil { } - /** - * isConditionAlwaysTrue - * - * @param expr - * @return - */ public static boolean isConditionAlwaysTrue(SQLExpr expr) { Object o = WallVisitorUtils.getValue(expr); return Boolean.TRUE.equals(o); } - /** - * isConditionAlwaysFalse - * - * @param expr - * @return - */ public static boolean isConditionAlwaysFalse(SQLExpr expr) { Object o = WallVisitorUtils.getValue(expr); return Boolean.FALSE.equals(o); } + + } diff --git a/src/main/java/com/actiontech/dble/server/handler/ExplainHandler.java b/src/main/java/com/actiontech/dble/server/handler/ExplainHandler.java index 3a6590447..289c10261 100644 --- a/src/main/java/com/actiontech/dble/server/handler/ExplainHandler.java +++ b/src/main/java/com/actiontech/dble/server/handler/ExplainHandler.java @@ -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; diff --git a/src/main/java/com/actiontech/dble/server/handler/SelectHandler.java b/src/main/java/com/actiontech/dble/server/handler/SelectHandler.java index a5bd04358..02a621133 100644 --- a/src/main/java/com/actiontech/dble/server/handler/SelectHandler.java +++ b/src/main/java/com/actiontech/dble/server/handler/SelectHandler.java @@ -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); diff --git a/src/main/java/com/actiontech/dble/server/handler/ServerLoadDataInfileHandler.java b/src/main/java/com/actiontech/dble/server/handler/ServerLoadDataInfileHandler.java index 4a3e9c3bf..3e21ea26f 100644 --- a/src/main/java/com/actiontech/dble/server/handler/ServerLoadDataInfileHandler.java +++ b/src/main/java/com/actiontech/dble/server/handler/ServerLoadDataInfileHandler.java @@ -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 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 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(); diff --git a/src/main/java/com/actiontech/dble/server/handler/SetHandler.java b/src/main/java/com/actiontech/dble/server/handler/SetHandler.java index 9fa91e2ee..cbb291446 100644 --- a/src/main/java/com/actiontech/dble/server/handler/SetHandler.java +++ b/src/main/java/com/actiontech/dble/server/handler/SetHandler.java @@ -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.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; diff --git a/src/main/java/com/actiontech/dble/server/parser/ServerParseSelect.java b/src/main/java/com/actiontech/dble/server/parser/ServerParseSelect.java index 250b6ddbf..7c8c8f426 100644 --- a/src/main/java/com/actiontech/dble/server/parser/ServerParseSelect.java +++ b/src/main/java/com/actiontech/dble/server/parser/ServerParseSelect.java @@ -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; } diff --git a/src/main/java/com/actiontech/dble/server/response/SelectTxReadOnly.java b/src/main/java/com/actiontech/dble/server/response/SelectTxReadOnly.java index e9e3fa441..b3dda4c8b 100644 --- a/src/main/java/com/actiontech/dble/server/response/SelectTxReadOnly.java +++ b/src/main/java/com/actiontech/dble/server/response/SelectTxReadOnly.java @@ -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); diff --git a/src/main/java/com/actiontech/dble/server/response/SelectVariables.java b/src/main/java/com/actiontech/dble/server/response/SelectVariables.java index 213188212..fc6d1bbd9 100644 --- a/src/main/java/com/actiontech/dble/server/response/SelectVariables.java +++ b/src/main/java/com/actiontech/dble/server/response/SelectVariables.java @@ -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"); } diff --git a/src/main/java/com/actiontech/dble/server/response/SessionIsolation.java b/src/main/java/com/actiontech/dble/server/response/SessionIsolation.java index c52d1cf58..925f5ee0f 100644 --- a/src/main/java/com/actiontech/dble/server/response/SessionIsolation.java +++ b/src/main/java/com/actiontech/dble/server/response/SessionIsolation.java @@ -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); diff --git a/src/main/java/com/actiontech/dble/server/variables/SystemVariables.java b/src/main/java/com/actiontech/dble/server/variables/SystemVariables.java index 06e3b0ad9..1aa84fdc3 100644 --- a/src/main/java/com/actiontech/dble/server/variables/SystemVariables.java +++ b/src/main/java/com/actiontech/dble/server/variables/SystemVariables.java @@ -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"); diff --git a/src/main/java/com/actiontech/dble/statistic/stat/SqlFrequency.java b/src/main/java/com/actiontech/dble/statistic/stat/SqlFrequency.java index a9828585b..432c67f87 100644 --- a/src/main/java/com/actiontech/dble/statistic/stat/SqlFrequency.java +++ b/src/main/java/com/actiontech/dble/statistic/stat/SqlFrequency.java @@ -57,6 +57,10 @@ public class SqlFrequency implements Comparable { 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 { @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; } diff --git a/src/main/java/com/actiontech/dble/statistic/stat/UserSqlHighStat.java b/src/main/java/com/actiontech/dble/statistic/stat/UserSqlHighStat.java index ec6364afa..207949d9b 100644 --- a/src/main/java/com/actiontech/dble/statistic/stat/UserSqlHighStat.java +++ b/src/main/java/com/actiontech/dble/statistic/stat/UserSqlHighStat.java @@ -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 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); } diff --git a/src/main/java/com/actiontech/dble/statistic/stat/UserStat.java b/src/main/java/com/actiontech/dble/statistic/stat/UserStat.java index b3b33908b..bcbd6cdd3 100644 --- a/src/main/java/com/actiontech/dble/statistic/stat/UserStat.java +++ b/src/main/java/com/actiontech/dble/statistic/stat/UserStat.java @@ -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 //----------------------------------------------------- diff --git a/src/test/java/com/actiontech/dble/buffer/TestDirectByteBufferPool.java b/src/test/java/com/actiontech/dble/buffer/TestDirectByteBufferPool.java index 1624b2d94..4d04804ce 100644 --- a/src/test/java/com/actiontech/dble/buffer/TestDirectByteBufferPool.java +++ b/src/test/java/com/actiontech/dble/buffer/TestDirectByteBufferPool.java @@ -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; diff --git a/src/test/java/com/actiontech/dble/cache/DefaultLayedCachePoolTest.java b/src/test/java/com/actiontech/dble/cache/DefaultLayedCachePoolTest.java index 934dea053..4a9add6f8 100644 --- a/src/test/java/com/actiontech/dble/cache/DefaultLayedCachePoolTest.java +++ b/src/test/java/com/actiontech/dble/cache/DefaultLayedCachePoolTest.java @@ -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 { diff --git a/src/test/java/com/actiontech/dble/cache/EnCachePoolTest.java b/src/test/java/com/actiontech/dble/cache/EnCachePoolTest.java index 7f51b9c11..c0831a7b0 100644 --- a/src/test/java/com/actiontech/dble/cache/EnCachePoolTest.java +++ b/src/test/java/com/actiontech/dble/cache/EnCachePoolTest.java @@ -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; diff --git a/src/test/java/com/actiontech/dble/parser/druid/ServerSchemaStatVisitorTest.java b/src/test/java/com/actiontech/dble/parser/druid/ServerSchemaStatVisitorTest.java index 0dad98e4d..5af73d6ef 100644 --- a/src/test/java/com/actiontech/dble/parser/druid/ServerSchemaStatVisitorTest.java +++ b/src/test/java/com/actiontech/dble/parser/druid/ServerSchemaStatVisitorTest.java @@ -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 = 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 whereUnits = getAllWhereUnit(sql); + WhereUnit whereUnit = whereUnits.get(0); + List 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> 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 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> 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 = getConditionList(sql); + List whereUnits = getAllWhereUnit(sql); + WhereUnit whereUnit = whereUnits.get(0); + List 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> 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 = getConditionList(sql); - + List whereUnits = getAllWhereUnit(sql); + List> 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> getConditionList(String sql) { - SQLStatementParser parser = null; - parser = new MySqlStatementParser(sql); + private List 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(); } diff --git a/src/test/java/com/actiontech/dble/performance/TestRandomDataUtil.java b/src/test/java/com/actiontech/dble/performance/TestRandomDataUtil.java index d10d62315..fed07f0b2 100644 --- a/src/test/java/com/actiontech/dble/performance/TestRandomDataUtil.java +++ b/src/test/java/com/actiontech/dble/performance/TestRandomDataUtil.java @@ -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; diff --git a/src/test/java/com/actiontech/dble/route/DDLRouteTest.java b/src/test/java/com/actiontech/dble/route/DDLRouteTest.java index 1a84d6d1b..2e7ab6970 100644 --- a/src/test/java/com/actiontech/dble/route/DDLRouteTest.java +++ b/src/test/java/com/actiontech/dble/route/DDLRouteTest.java @@ -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 diff --git a/src/test/java/com/actiontech/dble/route/DQLRouteTest.java b/src/test/java/com/actiontech/dble/route/DQLRouteTest.java deleted file mode 100644 index 6823ddb46..000000000 --- a/src/test/java/com/actiontech/dble/route/DQLRouteTest.java +++ /dev/null @@ -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 schemaMap; - protected LayerCachePool cachePool = new SimpleCachePool(); - protected RouteStrategy routeStrategy = RouteStrategyFactory.getRouteStrategy(); - private Map tableAliasMap = new HashMap(); - - 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 taskList = visitorParse(rrs, statement, visitor); - Assert.assertEquals(true, !taskList.get(0).getTablesAndConditions().isEmpty()); - } - - @SuppressWarnings("unchecked") - private List visitorParse(RouteResultset rrs, SQLStatement stmt, ServerSchemaStatVisitor visitor) throws Exception { - - stmt.accept(visitor); - - List> mergedConditionList = visitor.getConditionList(); - - if (visitor.getAliasMap() != null) { - for (Map.Entry 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 retList = new ArrayList(); - if (returnValue instanceof ArrayList) { - retList.add(((ArrayList) returnValue).get(0)); - //retList = (ArrayList)returnValue; - //System.out.println(taskList.get(0).getTablesAndConditions().values()); - } - return retList; - } - -} diff --git a/src/test/java/com/actiontech/dble/route/DruidMysqlCreateTableTest.java b/src/test/java/com/actiontech/dble/route/DruidMysqlCreateTableTest.java index 83c3110ff..85a74a065 100644 --- a/src/test/java/com/actiontech/dble/route/DruidMysqlCreateTableTest.java +++ b/src/test/java/com/actiontech/dble/route/DruidMysqlCreateTableTest.java @@ -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; diff --git a/src/test/java/com/actiontech/dble/route/DruidMysqlRouteStrategyTest.java b/src/test/java/com/actiontech/dble/route/DruidMysqlRouteStrategyTest.java index eddfbf521..88cc7ef9f 100644 --- a/src/test/java/com/actiontech/dble/route/DruidMysqlRouteStrategyTest.java +++ b/src/test/java/com/actiontech/dble/route/DruidMysqlRouteStrategyTest.java @@ -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 nodeSet = new HashSet(); 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 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 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 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 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 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 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)"); diff --git a/src/test/java/com/actiontech/dble/route/DruidMysqlSqlParserTest.java b/src/test/java/com/actiontech/dble/route/DruidMysqlSqlParserTest.java index dea3b0355..1baf3b661 100644 --- a/src/test/java/com/actiontech/dble/route/DruidMysqlSqlParserTest.java +++ b/src/test/java/com/actiontech/dble/route/DruidMysqlSqlParserTest.java @@ -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; diff --git a/src/test/java/com/actiontech/dble/route/RouteWithOrTest.java b/src/test/java/com/actiontech/dble/route/RouteWithOrTest.java deleted file mode 100644 index ee51dcfec..000000000 --- a/src/test/java/com/actiontech/dble/route/RouteWithOrTest.java +++ /dev/null @@ -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 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 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 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 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 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 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 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 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 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 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 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 tableAliasMap = (Map) format.invoke(parser, visitor.getAliasMap()); - - Method format2 = class1.getDeclaredMethod("buildRouteCalculateUnits", Map.class, List.class); - format2.setAccessible(true); - List result = (List) format2.invoke(parser, tableAliasMap, visitor.getConditionList()); - return result; - } catch (Exception e) { - throw new RuntimeException(e); - } - - } - -} diff --git a/src/test/java/com/actiontech/dble/route/TestSelectBetweenSqlParser.java b/src/test/java/com/actiontech/dble/route/TestSelectBetweenSqlParser.java index 112f64f6d..bcc851c9c 100644 --- a/src/test/java/com/actiontech/dble/route/TestSelectBetweenSqlParser.java +++ b/src/test/java/com/actiontech/dble/route/TestSelectBetweenSqlParser.java @@ -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; diff --git a/src/test/java/com/actiontech/dble/sequence/IncrSequenceZKHandlerTest.java b/src/test/java/com/actiontech/dble/sequence/IncrSequenceZKHandlerTest.java index cc595068a..88dfbc82c 100644 --- a/src/test/java/com/actiontech/dble/sequence/IncrSequenceZKHandlerTest.java +++ b/src/test/java/com/actiontech/dble/sequence/IncrSequenceZKHandlerTest.java @@ -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; diff --git a/src/test/java/com/actiontech/dble/sqlexecute/BaseSQLExeTest.java b/src/test/java/com/actiontech/dble/sqlexecute/BaseSQLExeTest.java index 0c16ae25a..22fc73e5c 100644 --- a/src/test/java/com/actiontech/dble/sqlexecute/BaseSQLExeTest.java +++ b/src/test/java/com/actiontech/dble/sqlexecute/BaseSQLExeTest.java @@ -5,7 +5,7 @@ */ package com.actiontech.dble.sqlexecute; -import junit.framework.Assert; +import org.junit.Assert; import java.sql.*; diff --git a/src/test/java/com/actiontech/dble/util/SmallSetTest.java b/src/test/java/com/actiontech/dble/util/SmallSetTest.java index 820093d9f..6a6776f7e 100644 --- a/src/test/java/com/actiontech/dble/util/SmallSetTest.java +++ b/src/test/java/com/actiontech/dble/util/SmallSetTest.java @@ -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; diff --git a/src/test/java/com/actiontech/dble/util/SplitUtilTest.java b/src/test/java/com/actiontech/dble/util/SplitUtilTest.java index f9ce923fd..c85affdc6 100644 --- a/src/test/java/com/actiontech/dble/util/SplitUtilTest.java +++ b/src/test/java/com/actiontech/dble/util/SplitUtilTest.java @@ -5,7 +5,7 @@ */ package com.actiontech.dble.util; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; /**