Merge branch '2.17.06.0/rel' into develop

This commit is contained in:
yanhuqing666
2017-07-05 16:52:50 +08:00
149 changed files with 1985 additions and 3084 deletions

2
.gitignore vendored
View File

@@ -120,3 +120,5 @@ src/main/resources/zkconf/schema.xml
src/main/resources/zkconf/rule.xml
conf/dnindex.properties
version.txt
copyResources.bat
dependency-reduced-pom.xml

View File

@@ -3,9 +3,9 @@ VERSION 2.17.06.0
CONTENT:
## 1.feature
1.移除writeType参数等效于原来writeType =0
2.conf/index_to_charset.properties的内容固化到代码
3.show @@binlog.status 显示节点间事务一致的binlog线
4.增强explain执行计划
2.conf/index_to_charset.properties的内容固化到代码 #77
3.show @@binlog.status 显示节点间事务一致的binlog线 #118
4.增强explain执行计划#143
5.show tables时,不显示未创建的表show full tables 拆分表的type会显示为SHARDING TABLEglobal表会显示为GLOBAL TABLE
6.ddl 执行前做一次心跳检查
7.多表查询中using()结果未合并重复列 #103
@@ -15,20 +15,81 @@ CONTENT:
2.自增序列部分算法缺陷及改进
3.单节点查询不加入主键缓存bug #160
4.租户权限不不隔离的问题 #164
5.show full tables from db当db为不存在的db时多发包导致乱序 #159
5.show full tables from db当db为不存在的db时多发包导致乱序 #159
6.数字范围算法(AutoPartitionByLong),有默认节点时between...and...可能路由错误 #145
## 3.不兼容项
配置
sequence_conf.properties 库表名要用"\`"包起来,用"."连接
sequence_db_conf.properties 库表名要用"\`"包起来,用"."连接
cacheservice.properties 库表名要用"\`"包起来,用"_"连接。
自增序列部分算法
A.本地时间戳方式
#### 3.1 全局序列配置
##### 3.1.1 sequence_conf.properties
库表名要用"\`"包起来,用"."连接
##### 3.1.2 sequence_db_conf.properties
库表名要用"\`"包起来,用"."连接
#### 3.2 缓存配置
3.2.1 cacheservice.properties 库表名要用"\`"包起来,用"_"连接。
#### 3.3 自增序列部分算法
##### 3.3.1.本地时间戳方式
ID= (30(毫秒时间戳前30位)+5(机器ID)+5(业务编码)+12(重复累加)+12(毫秒时间戳后12位)
B.分布式ZK ID 生成器
##### 3.3.2.分布式ZK ID 生成器
ID= 63位二进制 (9(线程ID) +5(实例ID)+4(机房ID)+6(重复累加) +39(毫秒求模,可用17年))
#### 3.4 server.xml
3.4.1 mycatNodeId 改为serverNodeId
------
## 4.ushard分支 #117
如果不需要混淆成字母类(eg:A.class),maven编译增加参数
-Dneed.obfuscate=false ,默认为true
#### 4.1 客户端登录信息
显示ushard相关字样
#### 4.2 注解
注解内部原本用mycat的改为ushard
#### 4.3 全局序列
数据库方式全局序列需要的dbseq.sql 已经重新生成mycat已经用ushard替换表结构相关名称改变
#### 4.4 配置
##### 4.4.1 xml的使用规约
mycat改为ushard ,例如server的根节点写法是
```
<!DOCTYPE ushard:server SYSTEM "server.dtd">
```
```
<ushard:server xmlns:ushard="http://io.ushard/">
```
##### 4.4.2 cacheservice.properties
此配置中需要指定类名用于反射加载缓存池类型,可使用简称
ehcache
leveldb
mapdb
来指代原有类名(不区分大小写)mycat分支下兼容原有方式和简称方式。
原有方式如下,按顺序与简称方式一一对应。
io.mycat.cache.impl.EnchachePooFactory
io.mycat.cache.impl.LevelDBCachePooFactory
io.mycat.cache.impl.MapDBCachePooFactory
和简称方式。
##### 4.4.3 rule.xml
rule.xml配置当中需要指定类名用于反射加载拆分算法可使用简称
Hash
StringHash
Enum
NumberRange
Date
PatternRange
来指代原有类名(不区分大小写)mycat分支下兼容原有方式和简称方式。
原有方式如下,按顺序与简称方式一一对应。
io.mycat.route.function.PartitionByLong(固定Hash 分区)
io.mycat.route.function.PartitionByString(String固定Hash 分区)
io.mycat.route.function.PartitionByFileMap(枚举方式)
io.mycat.route.function.AutoPartitionByLong(数字范围)
io.mycat.route.function.PartitionByDate(日期分区)
io.mycat.route.function.PartitionByPattern(取模范围约束)
#### 4.5 全局表检查
启用全局表检查时候列名由_mycat_op_time改为_ushard_op_time
#### 4.6 日志
日志路径logs/ushard.log。
里面涉及到的包名从io.mycat 改为com.actionsky.com
------
DATE:2017/04/20
VERSION 2.17.04.0

10
pom.xml
View File

@@ -268,9 +268,9 @@
<resources>
<resource>
<directory>src/main/resources</directory>
<excludes>
<exclude>**/.svn/**</exclude>
</excludes>
<includes>
<include>*.dtd</include>
</includes>
</resource>
<resource>
<directory>${basedir}</directory>
@@ -427,6 +427,10 @@
<token>@annotation-name@</token>
<value>mycat:</value>
</replacement>
<replacement>
<token>@root_prefix@</token>
<value>mycat</value>
</replacement>
</replacements>
</configuration>
</execution>

View File

@@ -0,0 +1,46 @@
syntax = "proto2";
package com.actionsky.ushard.meta.protocol;
option java_package = "com.actionsky.ushard.meta.protocol";
option java_outer_classname = "StructureMeta";
option optimize_for = SPEED;
message IndexMeta{
required string name = 1;
required string type = 2;//PRI,UNI,MUI
repeated string columns = 3;
}
message ColumnMeta{
required string name = 1;
required string dataType = 2;
optional bool isUnsigned = 3;
optional int32 length = 4;
optional int32 precision = 5;
optional int32 scale = 6;
optional bool canNull = 7;
optional string sdefault = 8;
optional bool auto_incre = 9;
optional string charsetname = 10 [default = "utf8"];
//[COMMENT 'string']
//[COLUMN_FORMAT {FIXED|DYNAMIC|DEFAULT}]
//[STORAGE {DISK|MEMORY|DEFAULT}]
//[reference_definition]
//[GENERATED ALWAYS] AS (expression)
}
message TableMeta{
optional string tableName = 1;//不带dbname的表名:如table1
repeated ColumnMeta columns = 2;//所有的列属性,有序
optional bool isView = 3[default = false];
optional string createSql = 4;
optional int64 version = 5;
optional int64 aiOffset = 6[default=-1];
optional int32 type = 7;
optional int32 aiColPos = 8[default=-1];
optional IndexMeta primary = 9;
repeated IndexMeta uniIndex = 10;
repeated IndexMeta index = 11;
}

View File

@@ -40,7 +40,6 @@ import io.mycat.config.MycatConfig;
import io.mycat.config.classloader.DynaClassLoader;
import io.mycat.config.loader.zkprocess.comm.ZkConfig;
import io.mycat.config.loader.zkprocess.comm.ZkParamCfg;
import io.mycat.config.loader.zkprocess.entity.server.System;
import io.mycat.config.model.SchemaConfig;
import io.mycat.config.model.SystemConfig;
import io.mycat.config.model.TableConfig;
@@ -82,14 +81,14 @@ import java.util.concurrent.atomic.AtomicLong;
*/
public class MycatServer {
public static final String NAME = "MyCat";
public static final String NAME = "MyCat_";
private static final long LOG_WATCH_DELAY = 60000L;
private static final long TIME_UPDATE_PERIOD = 20L;
private static final long DEFAULT_SQL_STAT_RECYCLE_PERIOD = 5 * 1000L;
private static final long DEFAULT_OLD_CONNECTION_CLEAR_PERIOD = 5 * 1000L;
private static final MycatServer INSTANCE = new MycatServer();
private static final Logger LOGGER = LoggerFactory.getLogger("MycatServer");
private static final Logger LOGGER = LoggerFactory.getLogger("Server");
private static final Repository fileRepository = new FileSystemRepository();
private AtomicBoolean backupLocked;
@@ -119,7 +118,7 @@ public class MycatServer {
/**
* Mycat 内存管理类
*/
private MyCatMemory myCatMemory = null;
private MyCatMemory serverMemory = null;
public static final MycatServer getInstance() {
return INSTANCE;
@@ -218,7 +217,7 @@ public class MycatServer {
seq = xaIDInc.incrementAndGet();
}
}
return "'Mycat." + this.getConfig().getSystem().getMycatNodeId() + "." + seq + "'";
return "'"+NAME+"Server." + this.getConfig().getSystem().getServerNodeId() + "." + seq + "'";
}
private void genXidSeq(String xaID) {
@@ -243,8 +242,8 @@ public class MycatServer {
throw new java.lang.IllegalArgumentException("Invalid sequnce handler type " + seqHandlerType);
}
}
public MyCatMemory getMyCatMemory() {
return myCatMemory;
public MyCatMemory getServerMemory() {
return serverMemory;
}
public XASessionCheck getXaSessionCheck() {
@@ -288,7 +287,7 @@ public class MycatServer {
// server startup
LOGGER.info("===============================================");
LOGGER.info(NAME + " is ready to startup ...");
LOGGER.info(NAME + "Server is ready to startup ...");
String inf = "Startup processors ...,total processors:"
+ system.getProcessors() + ",aio thread pool size:"
+ system.getProcessorExecutor()
@@ -330,7 +329,7 @@ public class MycatServer {
*/
if(system.getUseOffHeapForMerge() == 1){
try {
myCatMemory = new MyCatMemory(system,totalNetWorkBufferSize);
serverMemory = new MyCatMemory(system,totalNetWorkBufferSize);
} catch (NoSuchFieldException e) {
LOGGER .error("NoSuchFieldException",e);
} catch (IllegalAccessException e) {

View File

@@ -28,6 +28,7 @@ package io.mycat;
import java.text.SimpleDateFormat;
import java.util.Date;
import io.mycat.manager.handler.ShowServerLog;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -55,7 +56,7 @@ public final class MycatStartup {
// startup
server.startup();
System.out.println("MyCAT Server startup successfully. see logs in logs/mycat.log");
System.out.println("Server startup successfully. see logs in logs/"+ ShowServerLog.DEFAULT_LOGFILE);
} catch (Exception e) {
SimpleDateFormat sdf = new SimpleDateFormat(dateFormat);

View File

@@ -55,7 +55,7 @@ public class MySQLConsistencyChecker{
private String[] physicalSchemas;
// private String columnExistSQL = "select count(*) as "+GlobalTableUtil.INNER_COLUMN
// + " from information_schema.columns where column_name='"
// + GlobalTableUtil.GLOBAL_TABLE_MYCAT_COLUMN + "' and table_name='";
// + GlobalTableUtil.GLOBAL_TABLE_CHECK_COLUMN + "' and table_name='";
// 此处用到了 mysql 多行转一行 group_concat 的用法,主要是为了简化对结果的处理
// 得到的结果类似于id,name,_mycat_op_time
@@ -72,7 +72,7 @@ public class MySQLConsistencyChecker{
this.tableName = tableName;
this.countSQL = " select count(*) as "+GlobalTableUtil.COUNT_COLUMN+" from "
+ this.tableName;
this.maxSQL = " select max("+GlobalTableUtil.GLOBAL_TABLE_MYCAT_COLUMN+") as "+
this.maxSQL = " select max("+GlobalTableUtil.GLOBAL_TABLE_CHECK_COLUMN +") as "+
GlobalTableUtil.MAX_COLUMN+" from " + this.tableName;
this.columnExistSQL += this.tableName +"' ";
}

View File

@@ -157,6 +157,7 @@ public class SingleNodeHandler implements ResponseHandler, LoadDataResponseHandl
}
@Override
public void connectionError(Throwable e, BackendConnection conn) {
session.handleSpecial(rrs,session.getSource().getSchema(), true);
recycleResources();
session.getSource().close(e.getMessage());
}

View File

@@ -50,7 +50,7 @@ abstract class BaseHandlerBuilder {
/* 当前的最后一个handler */
protected DMLResponseHandler currentLast;
private PlanNode node;
protected MycatConfig mycatConfig;
protected MycatConfig config;
/* 是否可以全下推 */
protected boolean canPushDown = false;
/* 是否需要common中的handler包括group byorder bylimit等 */
@@ -66,8 +66,8 @@ abstract class BaseHandlerBuilder {
: MySQLNodeType.SLAVE;
this.node = node;
this.hBuilder = hBuilder;
this.mycatConfig = MycatServer.getInstance().getConfig();
if (mycatConfig.getSchemas().isEmpty())
this.config = MycatServer.getInstance().getConfig();
if (config.getSchemas().isEmpty())
throw new MySQLOutPutException(ErrorCode.ER_QUERYHANDLER, "", "current router config is empty!");
}
@@ -410,7 +410,7 @@ abstract class BaseHandlerBuilder {
}
protected TableConfig getTableConfig(String schema, String table) {
SchemaConfig schemaConfig = this.mycatConfig.getSchemas().get(schema);
SchemaConfig schemaConfig = this.config.getSchemas().get(schema);
if (schemaConfig == null)
return null;
return schemaConfig.getTables().get(table);

View File

@@ -89,21 +89,18 @@ public class HandlerBuilder {
}
private BaseHandlerBuilder createBuilder(final NonBlockingSession session, PlanNode node, HandlerBuilder context) {
switch (node.type()) {
case TABLE: {
PlanNode.PlanNodeType i = node.type();
if (i == PlanNode.PlanNodeType.TABLE) {
return new TableNodeHandlerBuilder(session, (TableNode) node, this);
}
case JOIN: {
} else if (i == PlanNode.PlanNodeType.JOIN) {
return new JoinNodeHandlerBuilder(session, (JoinNode) node, this);
}
case MERGE: {
} else if (i == PlanNode.PlanNodeType.MERGE) {
return new MergeNodeHandlerBuilder(session, (MergeNode) node, this);
}
case QUERY:
} else if (i == PlanNode.PlanNodeType.QUERY) {
return new QueryNodeHandlerBuilder(session, (QueryNode) node, this);
case NONAME:
} else if (i == PlanNode.PlanNodeType.NONAME) {
return new NoNameNodeHandlerBuilder(session, (NoNameNode) node, this);
default:
} else {
}
throw new RuntimeException("not supported tree node type:" + node.type());
}

View File

@@ -62,8 +62,7 @@ class JoinNodeHandlerBuilder extends BaseHandlerBuilder {
List<DMLResponseHandler> pres = new ArrayList<DMLResponseHandler>();
PlanNode left = node.getLeftNode();
PlanNode right = node.getRightNode();
switch (node.getStrategy()) {
case NESTLOOP:
if (node.getStrategy() == JoinNode.Strategy.NESTLOOP) {
final boolean isLeftSmall = left.getNestLoopFilters() == null;
final PlanNode tnSmall = isLeftSmall ? left : right;
final PlanNode tnBig = isLeftSmall ? right : left;
@@ -100,15 +99,15 @@ class JoinNodeHandlerBuilder extends BaseHandlerBuilder {
}
};
tempHandler.setTempDoneCallBack(tempDone);
break;
case SORTMERGE:
} else if (node.getStrategy() == JoinNode.Strategy.SORTMERGE) {
DMLResponseHandler lh = buildJoinChild(left, true);
pres.add(lh);
DMLResponseHandler rh = buildJoinChild(right, false);
pres.add(rh);
break;
default:
throw new MySQLOutPutException(ErrorCode.ER_QUERYHANDLER, "","strategy ["+node.getStrategy()+"] not implement yet!" );
} else {
throw new MySQLOutPutException(ErrorCode.ER_QUERYHANDLER, "", "strategy [" + node.getStrategy() + "] not implement yet!");
}
return pres;
}

View File

@@ -29,7 +29,7 @@ public class MergeBuilder {
private NonBlockingSession session;
private MySQLNodeType nodeType;
private String schema;
private MycatConfig mycatConfig;
private MycatConfig config;
private PushDownVisitor pdVisitor;
public MergeBuilder(NonBlockingSession session, PlanNode node, boolean needCommon, boolean needSendMaker,
@@ -41,7 +41,7 @@ public class MergeBuilder {
this.schema = session.getSource().getSchema();
this.nodeType = session.getSource().isTxstart() || !session.getSource().isAutocommit() ? MySQLNodeType.MASTER
: MySQLNodeType.SLAVE;
this.mycatConfig = MycatServer.getInstance().getConfig();
this.config = MycatServer.getInstance().getConfig();
this.pdVisitor = pdVisitor;
}
@@ -62,7 +62,7 @@ public class MergeBuilder {
RouteResultset rrs = new RouteResultset(sql, ServerParse.SELECT);
LayerCachePool pool = MycatServer.getInstance().getRouterservice().getTableId2DataNodeCache();
SchemaConfig schemaConfig = mycatConfig.getSchemas().get(node.getReferedTableNodes().get(0).getSchema());
SchemaConfig schemaConfig = config.getSchemas().get(node.getReferedTableNodes().get(0).getSchema());
return RouterUtil.routeFromParser(druidParser, schemaConfig, rrs, select, sql, pool, visitor, session.getSource());
}

View File

@@ -39,7 +39,7 @@ class NoNameNodeHandlerBuilder extends BaseHandlerBuilder {
this.canPushDown = true;
String sql = vistor.getSql().toString();
String schema = session.getSource().getSchema();
SchemaConfig schemacfg = mycatConfig.getSchemas().get(schema);
SchemaConfig schemacfg = config.getSchemas().get(schema);
RouteResultsetNode[] rrss = getTableSources(schemacfg.getAllDataNodes(),sql);
hBuilder.checkRRSS(rrss);
MultiNodeMergeHandler mh = new MultiNodeMergeHandler(getSequenceId(), rrss, session.getSource().isAutocommit()&&!session.getSource().isTxstart(),

View File

@@ -27,23 +27,23 @@ public class GlobalVisitor extends MysqlVisitor {
if (!visited) {
replaceableSqlBuilder.clear();
sqlBuilder = replaceableSqlBuilder.getCurrentElement().getSb();
switch (query.type()) {
case TABLE:
PlanNodeType i = query.type();
if (i == PlanNodeType.TABLE) {
visit((TableNode) query);
break;
case JOIN:
} else if (i == PlanNodeType.JOIN) {
visit((JoinNode) query);
break;
case QUERY:
} else if (i == PlanNodeType.QUERY) {
visit((QueryNode) query);
break;
case MERGE:
} else if (i == PlanNodeType.MERGE) {
visit((MergeNode) query);
break;
case NONAME:
} else if (i == PlanNodeType.NONAME) {
visit((NoNameNode) query);
break;
default:
} else {
throw new RuntimeException("not implement yet!");
}
visited = true;

View File

@@ -38,14 +38,14 @@ public class PushDownVisitor extends MysqlVisitor {
replaceableSqlBuilder.clear();
sqlBuilder = replaceableSqlBuilder.getCurrentElement().getSb();
// 在已经visited的情况下pushdownvisitor只要进行table名称的替换即可
switch (query.type()) {
case TABLE:
PlanNode.PlanNodeType i = query.type();
if (i == PlanNode.PlanNodeType.TABLE) {
visit((TableNode) query);
break;
case JOIN:
} else if (i == PlanNode.PlanNodeType.JOIN) {
visit((JoinNode) query);
break;
default:
} else {
throw new RuntimeException("not implement yet!");
}
visited = true;
@@ -170,8 +170,8 @@ public class PushDownVisitor extends MysqlVisitor {
ItemSum funCol = (ItemSum) col;
String funName = funCol.funcName().toUpperCase();
String colName = pdName;
switch (funCol.sumType()) {
case AVG_FUNC: {
ItemSum.Sumfunctype i = funCol.sumType();
if (i == ItemSum.Sumfunctype.AVG_FUNC) {
String colNameSum = colName.replace(funName + "(", "SUM(");
colNameSum = colNameSum.replace(getMadeAggAlias(funName), getMadeAggAlias("SUM"));
String colNameCount = colName.replace(funName + "(", "COUNT(");
@@ -181,18 +181,14 @@ public class PushDownVisitor extends MysqlVisitor {
pushDownTableInfos.add(null);
pushDownTableInfos.add(null);
}
}
continue;
case STD_FUNC:
case VARIANCE_FUNC: {
// variance:下发时 v[0]:count,v[1]:sum,v[2]:variance(局部)
} else if (i == ItemSum.Sumfunctype.STD_FUNC || i == ItemSum.Sumfunctype.VARIANCE_FUNC) {
String colNameCount = colName.replace(funName + "(", "COUNT(");
colNameCount = colNameCount.replace(getMadeAggAlias(funName), getMadeAggAlias("COUNT"));
String colNameSum = colName.replace(funName + "(", "SUM(");
colNameSum = colNameSum.replace(getMadeAggAlias(funName), getMadeAggAlias("SUM"));
String colNameVar = colName.replace(funName + "(", "VARIANCE(");
colNameVar = colNameVar.replace(getMadeAggAlias(funName), getMadeAggAlias("VARIANCE"));
// VARIANCE
sqlBuilder.append(colNameCount).append(",").append(colNameSum).append(",").append(colNameVar)
.append(",");
if (addPushDownTableInfo) {
@@ -200,10 +196,7 @@ public class PushDownVisitor extends MysqlVisitor {
pushDownTableInfos.add(null);
pushDownTableInfos.add(null);
}
}
continue;
default:
break;
}
}
sqlBuilder.append(pdName);

View File

@@ -30,8 +30,8 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
}
@Override
protected boolean executeCommit(MySQLConnection mysqlCon, int position) {
switch (session.getXaState()) {
case TX_STARTED_STATE:
TxState state = session.getXaState();
if(state == TxState.TX_STARTED_STATE) {
if (participantLogEntry == null) {
participantLogEntry = new ParticipantLogEntry[nodeCount];
CoordinatorLogEntry coordinatorLogEntry = new CoordinatorLogEntry(session.getSessionXaID(), participantLogEntry, session.getXaState());
@@ -39,8 +39,7 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
}
XAStateLog.initRecoverylog(session.getSessionXaID(), position, mysqlCon);
endPhase(mysqlCon);
break;
case TX_ENDED_STATE:
}else if (state == TxState.TX_ENDED_STATE) {
if (position == 0) {
if (!XAStateLog.saveXARecoverylog(session.getSessionXaID(), TxState.TX_PREPARING_STATE)) {
String errMsg = "saveXARecoverylog error, the stage is TX_PREPARING_STATE";
@@ -51,8 +50,7 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
}
}
preparePhase(mysqlCon);
break;
case TX_PREPARED_STATE:
} else if(state == TxState.TX_PREPARED_STATE) {
if (position == 0) {
if (!XAStateLog.saveXARecoverylog(session.getSessionXaID(), TxState.TX_COMMITING_STATE)) {
String errMsg = "saveXARecoverylog error, the stage is TX_COMMITING_STATE";
@@ -63,14 +61,11 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
}
}
commitPhase(mysqlCon);
break;
case TX_COMMIT_FAILED_STATE:
} else if(state == TxState.TX_COMMIT_FAILED_STATE) {
if (position == 0) {
XAStateLog.saveXARecoverylog(session.getSessionXaID(), TxState.TX_COMMIT_FAILED_STATE);
}
commitPhase(mysqlCon);
break;
default:
}
return true;
}
@@ -108,30 +103,26 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
@Override
public void okResponse(byte[] ok, BackendConnection conn) {
MySQLConnection mysqlCon = (MySQLConnection) conn;
switch (mysqlCon.getXaStatus()) {
// END OK
case TX_STARTED_STATE:
TxState state =mysqlCon.getXaStatus();
if(state == TxState.TX_STARTED_STATE) {
mysqlCon.setXaStatus(TxState.TX_ENDED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
if (decrementCountBy(1)) {
session.setXaState(TxState.TX_ENDED_STATE);
nextParse();
}
break;
//PREPARE OK
case TX_ENDED_STATE:
} else if(state == TxState.TX_ENDED_STATE) {
//PREPARE OK
mysqlCon.setXaStatus(TxState.TX_PREPARED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
if (decrementCountBy(1)) {
if(session.getXaState()== TxState.TX_ENDED_STATE){
if (session.getXaState() == TxState.TX_ENDED_STATE) {
session.setXaState(TxState.TX_PREPARED_STATE);
}
nextParse();
}
break;
//COMMIT OK
case TX_COMMIT_FAILED_STATE:
case TX_PREPARED_STATE:
}else if(state == TxState.TX_COMMIT_FAILED_STATE ||state == TxState.TX_PREPARED_STATE) {
//COMMIT OK
// XA reset status now
mysqlCon.setXaStatus(TxState.TX_COMMITED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -142,8 +133,6 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
}
cleanAndFeedback();
}
break;
default:
// LOGGER.error("Wrong XA status flag!");
}
}
@@ -157,9 +146,7 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
sendData = makeErrorPacket(errmsg);
if (conn instanceof MySQLConnection) {
MySQLConnection mysqlCon = (MySQLConnection) conn;
switch (mysqlCon.getXaStatus()) {
// 'xa end' error
case TX_STARTED_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_STARTED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_CONN_QUIT);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -167,32 +154,27 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
session.setXaState(TxState.TX_ENDED_STATE);
nextParse();
}
break;
// 'xa prepare' error
case TX_ENDED_STATE:
// 'xa prepare' error
} else if (mysqlCon.getXaStatus() == TxState.TX_ENDED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_CONN_QUIT);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
if (decrementCountBy(1)) {
if(session.getXaState()== TxState.TX_ENDED_STATE){
if (session.getXaState() == TxState.TX_ENDED_STATE) {
session.setXaState(TxState.TX_PREPARED_STATE);
}
nextParse();
}
break;
// 'xa commit' err
case TX_COMMIT_FAILED_STATE:
case TX_PREPARED_STATE:
//TODO :服务降级?
// 'xa commit' err
} else if (mysqlCon.getXaStatus() == TxState.TX_COMMIT_FAILED_STATE || mysqlCon.getXaStatus() == TxState.TX_PREPARED_STATE) {//TODO :服务降级?
mysqlCon.setXaStatus(TxState.TX_COMMIT_FAILED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
session.setXaState(TxState.TX_COMMIT_FAILED_STATE);
if (decrementCountBy(1)) {
cleanAndFeedback();
}
break;
default:
// LOGGER.error("Wrong XA status flag!");
}
}
}
@@ -205,9 +187,7 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
sendData = makeErrorPacket(errmsg);
if (conn instanceof MySQLConnection) {
MySQLConnection mysqlCon = (MySQLConnection) conn;
switch (mysqlCon.getXaStatus()) {
// 'xa end' connectionError
case TX_STARTED_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_STARTED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_CONN_QUIT);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -215,29 +195,24 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
session.setXaState(TxState.TX_ENDED_STATE);
nextParse();
}
break;
// 'xa prepare' connectionError
case TX_ENDED_STATE:
// 'xa prepare' connectionError
} else if (mysqlCon.getXaStatus() == TxState.TX_ENDED_STATE) {
mysqlCon.setXaStatus(TxState.TX_PREPARE_UNCONNECT_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
session.setXaState(TxState.TX_PREPARE_UNCONNECT_STATE);
if (decrementCountBy(1)) {
nextParse();
}
break;
// 'xa commit' connectionError
case TX_COMMIT_FAILED_STATE:
case TX_PREPARED_STATE:
//TODO :服务降级?
// 'xa commit' connectionError
} else if (mysqlCon.getXaStatus() == TxState.TX_COMMIT_FAILED_STATE || mysqlCon.getXaStatus() == TxState.TX_PREPARED_STATE) {//TODO :服务降级?
mysqlCon.setXaStatus(TxState.TX_COMMIT_FAILED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
session.setXaState(TxState.TX_COMMIT_FAILED_STATE);
if (decrementCountBy(1)) {
cleanAndFeedback();
}
break;
default:
// LOGGER.error("Wrong XA status flag!");
}
}
}
@@ -248,9 +223,7 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
sendData = makeErrorPacket(reason);
if (conn instanceof MySQLConnection) {
MySQLConnection mysqlCon = (MySQLConnection) conn;
switch (mysqlCon.getXaStatus()) {
// 'xa end' connectionClose,conn has quit
case TX_STARTED_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_STARTED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_CONN_QUIT);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -258,29 +231,24 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
session.setXaState(TxState.TX_ENDED_STATE);
nextParse();
}
break;
// 'xa prepare' connectionClose,conn has quit
case TX_ENDED_STATE:
// 'xa prepare' connectionClose,conn has quit
} else if (mysqlCon.getXaStatus() == TxState.TX_ENDED_STATE) {
mysqlCon.setXaStatus(TxState.TX_PREPARE_UNCONNECT_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
session.setXaState(TxState.TX_PREPARE_UNCONNECT_STATE);
if (decrementCountBy(1)) {
nextParse();
}
break;
// 'xa commit' connectionClose
case TX_COMMIT_FAILED_STATE:
case TX_PREPARED_STATE:
//TODO :服务降级?
} else if (mysqlCon.getXaStatus() == TxState.TX_COMMIT_FAILED_STATE || mysqlCon.getXaStatus() == TxState.TX_PREPARED_STATE) {//TODO :服务降级?
mysqlCon.setXaStatus(TxState.TX_COMMIT_FAILED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
session.setXaState(TxState.TX_COMMIT_FAILED_STATE);
if (decrementCountBy(1)) {
cleanAndFeedback();
}
break;
default:
// LOGGER.error("Wrong XA status flag!");
}
}
}
@@ -295,9 +263,7 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
}
}
private void cleanAndFeedback() {
switch (session.getXaState()) {
case TX_INITIALIZE_STATE:
// clear all resources
if (session.getXaState() == TxState.TX_INITIALIZE_STATE) {// clear all resources
XAStateLog.saveXARecoverylog(session.getSessionXaID(), TxState.TX_COMMITED_STATE);
//在这里释放取消限制锁
session.cancelableStatusSet(NonBlockingSession.CANCEL_STATUS_INIT);
@@ -307,9 +273,9 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
return;
}
session.getSource().write(send);
break;
// partitionly commited,must commit again
case TX_COMMIT_FAILED_STATE:
// partitionly commited,must commit again
} else if (session.getXaState() == TxState.TX_COMMIT_FAILED_STATE) {
MySQLConnection errConn = session.releaseExcept(TxState.TX_COMMIT_FAILED_STATE);
if (errConn != null) {
XAStateLog.saveXARecoverylog(session.getSessionXaID(), session.getXaState());
@@ -326,19 +292,19 @@ public class XACommitNodesHandler extends AbstractCommitNodesHandler {
session.setXaState(TxState.TX_INITIALIZE_STATE);
//在这里释放取消限制锁
session.cancelableStatusSet(NonBlockingSession.CANCEL_STATUS_INIT);
byte[]toSend = sendData;
byte[] toSend = sendData;
session.clearResources(false);
if (!session.closed()) {
session.getSource().write(toSend);
}
}
break;
// need to rollback;
default:
// need to rollback;
} else {
XAStateLog.saveXARecoverylog(session.getSessionXaID(), session.getXaState());
session.getSource().write(sendData);
LOGGER.warn("cleanAndFeedback:" + error);
break;
}
}
}

View File

@@ -85,82 +85,73 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
if(position==0 && participantLogEntry != null){
XAStateLog.saveXARecoverylog(session.getSessionXaID(), session.getXaState());
}
switch (session.getXaState()) {
case TX_STARTED_STATE:
if (session.getXaState() == TxState.TX_STARTED_STATE) {
if (participantLogEntry == null) {
participantLogEntry = new ParticipantLogEntry[nodeCount];
CoordinatorLogEntry coordinatorLogEntry = new CoordinatorLogEntry(session.getSessionXaID(), participantLogEntry, session.getXaState());
XAStateLog.flushMemoryRepository(session.getSessionXaID(), coordinatorLogEntry);
}
XAStateLog.initRecoverylog(session.getSessionXaID(), position, mysqlCon);
if(mysqlCon.isClosed()){
if (mysqlCon.isClosed()) {
mysqlCon.setXaStatus(TxState.TX_CONN_QUIT);
}
endPhase(mysqlCon);
break;
case TX_PREPARED_STATE:
} else if (session.getXaState() == TxState.TX_PREPARED_STATE) {
if (position == 0) {
if(!XAStateLog.saveXARecoverylog(session.getSessionXaID(), TxState.TX_ROLLBACKING_STATE)){
if (!XAStateLog.saveXARecoverylog(session.getSessionXaID(), TxState.TX_ROLLBACKING_STATE)) {
this.setFail("saveXARecoverylog error, the stage is TX_ROLLBACKING_STATE");
cleanAndFeedback();
return false;
}
}
case TX_ROLLBACK_FAILED_STATE:
case TX_PREPARE_UNCONNECT_STATE:
rollbackPhase(mysqlCon);
break;
case TX_ENDED_STATE:
if(mysqlCon.isClosed()){
} else if (session.getXaState() == TxState.TX_ROLLBACK_FAILED_STATE || session.getXaState() == TxState.TX_PREPARE_UNCONNECT_STATE) {
rollbackPhase(mysqlCon);
} else if (session.getXaState() == TxState.TX_ENDED_STATE) {
if (mysqlCon.isClosed()) {
mysqlCon.setXaStatus(TxState.TX_CONN_QUIT);
}
rollbackPhase(mysqlCon);
break;
default:
}
return true;
}
private void endPhase(MySQLConnection mysqlCon) {
switch (mysqlCon.getXaStatus()) {
case TX_STARTED_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_STARTED_STATE) {
String xaTxId = mysqlCon.getConnXID(session);
mysqlCon.execCmd("XA END " + xaTxId + ";");
break;
case TX_CONN_QUIT:
} else if (mysqlCon.getXaStatus() == TxState.TX_CONN_QUIT) {
if (decrementCountBy(1)) {
session.setXaState(TxState.TX_ENDED_STATE);
rollback();
}
default:
break;
}
}
private void rollbackPhase(MySQLConnection mysqlCon) {
switch (mysqlCon.getXaStatus()) {
case TX_ROLLBACK_FAILED_STATE:
case TX_PREPARE_UNCONNECT_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_ROLLBACK_FAILED_STATE || mysqlCon.getXaStatus() == TxState.TX_PREPARE_UNCONNECT_STATE) {
MySQLConnection newConn = session.freshConn(mysqlCon, this);
if (!newConn.equals(mysqlCon)) {
mysqlCon = newConn;
String xaTxId = mysqlCon.getConnXID(session);
mysqlCon.execCmd("XA ROLLBACK " + xaTxId + ";");
} else if (decrementCountBy(1)) {
cleanAndFeedback();
break;
}
case TX_ENDED_STATE:
case TX_PREPARED_STATE:
} else if (mysqlCon.getXaStatus() == TxState.TX_ENDED_STATE || mysqlCon.getXaStatus() == TxState.TX_PREPARED_STATE) {
String xaTxId = mysqlCon.getConnXID(session);
mysqlCon.execCmd("XA ROLLBACK " + xaTxId + ";");
break;
case TX_CONN_QUIT:
case TX_ROLLBACKED_STATE:
} else if (mysqlCon.getXaStatus() == TxState.TX_CONN_QUIT || mysqlCon.getXaStatus() == TxState.TX_ROLLBACKED_STATE) {
if (decrementCountBy(1)) {
cleanAndFeedback();
}
break;
default:
break;
}
}
@@ -168,18 +159,16 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
public void okResponse(byte[] ok, BackendConnection conn) {
if (conn instanceof MySQLConnection) {
MySQLConnection mysqlCon = (MySQLConnection) conn;
switch (mysqlCon.getXaStatus()) {
// 'xa end' ok
case TX_STARTED_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_STARTED_STATE) {
mysqlCon.setXaStatus(TxState.TX_ENDED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
if (decrementCountBy(1)) {
session.setXaState(TxState.TX_ENDED_STATE);
rollback();
}
break;
// 'xa rollback' ok without prepared
case TX_ENDED_STATE:
// 'xa rollback' ok without prepared
} else if (mysqlCon.getXaStatus() == TxState.TX_ENDED_STATE) {
mysqlCon.setXaStatus(TxState.TX_ROLLBACKED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
mysqlCon.setXaStatus(TxState.TX_INITIALIZE_STATE);
@@ -187,24 +176,22 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
session.setXaState(TxState.TX_INITIALIZE_STATE);
cleanAndFeedback();
}
break;
// 'xa rollback' ok
case TX_PREPARED_STATE:
// we dont' konw if the conn prepared or not
case TX_PREPARE_UNCONNECT_STATE:
case TX_ROLLBACK_FAILED_STATE:
// 'xa rollback' ok
} else if (mysqlCon.getXaStatus() == TxState.TX_PREPARED_STATE || mysqlCon.getXaStatus() == TxState.TX_PREPARE_UNCONNECT_STATE || mysqlCon.getXaStatus() == TxState.TX_ROLLBACK_FAILED_STATE) {// we dont' konw if the conn prepared or not
mysqlCon.setXaStatus(TxState.TX_ROLLBACKED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
mysqlCon.setXaStatus(TxState.TX_INITIALIZE_STATE);
if (decrementCountBy(1)) {
if(session.getXaState()== TxState.TX_PREPARED_STATE){
if (session.getXaState() == TxState.TX_PREPARED_STATE) {
session.setXaState(TxState.TX_INITIALIZE_STATE);
}
cleanAndFeedback();
}
break;
default:
// LOGGER.error("Wrong XA status flag!");
} else {// LOGGER.error("Wrong XA status flag!");
}
}
}
@@ -213,9 +200,7 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
public void errorResponse(byte[] err, BackendConnection conn) {
if (conn instanceof MySQLConnection) {
MySQLConnection mysqlCon = (MySQLConnection) conn;
switch (mysqlCon.getXaStatus()) {
// 'xa end' err
case TX_STARTED_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_STARTED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_ROLLBACKED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -223,9 +208,9 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
session.setXaState(TxState.TX_ENDED_STATE);
rollback();
}
break;
// 'xa rollback' ok without prepared
case TX_ENDED_STATE:
// 'xa rollback' ok without prepared
} else if (mysqlCon.getXaStatus() == TxState.TX_ENDED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_ROLLBACKED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -233,10 +218,9 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
session.setXaState(TxState.TX_INITIALIZE_STATE);
cleanAndFeedback();
}
break;
// 'xa rollback' err
case TX_ROLLBACK_FAILED_STATE:
case TX_PREPARED_STATE:
// 'xa rollback' err
} else if (mysqlCon.getXaStatus() == TxState.TX_ROLLBACK_FAILED_STATE || mysqlCon.getXaStatus() == TxState.TX_PREPARED_STATE) {
mysqlCon.setXaStatus(TxState.TX_ROLLBACK_FAILED_STATE);
mysqlCon.quit();
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -244,9 +228,9 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
if (decrementCountBy(1)) {
cleanAndFeedback();
}
break;
// we dont' konw if the conn prepared or not
case TX_PREPARE_UNCONNECT_STATE:
// we dont' konw if the conn prepared or not
} else if (mysqlCon.getXaStatus() == TxState.TX_PREPARE_UNCONNECT_STATE) {
ErrorPacket errPacket = new ErrorPacket();
errPacket.read(err);
if (errPacket.errno == ErrorCode.ER_XAER_NOTA) {
@@ -267,9 +251,8 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
cleanAndFeedback();
}
}
break;
default:
// LOGGER.error("Wrong XA status flag!");
} else {// LOGGER.error("Wrong XA status flag!");
}
}
}
@@ -278,9 +261,7 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
public void connectionError(Throwable e, BackendConnection conn) {
if (conn instanceof MySQLConnection) {
MySQLConnection mysqlCon = (MySQLConnection) conn;
switch (mysqlCon.getXaStatus()) {
// 'xa end' connectionError
case TX_STARTED_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_STARTED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_ROLLBACKED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -288,8 +269,9 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
session.setXaState(TxState.TX_ENDED_STATE);
rollback();
}
// 'xa rollback' ok without prepared
case TX_ENDED_STATE:
// 'xa rollback' ok without prepared
} else if (mysqlCon.getXaStatus() == TxState.TX_ENDED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_ROLLBACKED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -297,9 +279,9 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
session.setXaState(TxState.TX_INITIALIZE_STATE);
cleanAndFeedback();
}
break;
// 'xa rollback' err
case TX_PREPARED_STATE:
// 'xa rollback' err
} else if (mysqlCon.getXaStatus() == TxState.TX_PREPARED_STATE) {
mysqlCon.setXaStatus(TxState.TX_ROLLBACK_FAILED_STATE);
mysqlCon.quit();
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -307,17 +289,16 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
if (decrementCountBy(1)) {
cleanAndFeedback();
}
break;
// we dont' konw if the conn prepared or not
case TX_PREPARE_UNCONNECT_STATE:
// we dont' konw if the conn prepared or not
} else if (mysqlCon.getXaStatus() == TxState.TX_PREPARE_UNCONNECT_STATE) {
session.setXaState(TxState.TX_ROLLBACK_FAILED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
if (decrementCountBy(1)) {
cleanAndFeedback();
}
break;
default:
// LOGGER.error("Wrong XA status flag!");
} else {// LOGGER.error("Wrong XA status flag!");
}
}
}
@@ -327,9 +308,7 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
this.setFail(reason);
if (conn instanceof MySQLConnection) {
MySQLConnection mysqlCon = (MySQLConnection) conn;
switch (mysqlCon.getXaStatus()) {
// 'xa end' err
case TX_STARTED_STATE:
if (mysqlCon.getXaStatus() == TxState.TX_STARTED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_ROLLBACKED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -337,8 +316,9 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
session.setXaState(TxState.TX_ENDED_STATE);
rollback();
}
// 'xa rollback' ok without prepared
case TX_ENDED_STATE:
// 'xa rollback' ok without prepared
} else if (mysqlCon.getXaStatus() == TxState.TX_ENDED_STATE) {
mysqlCon.quit();
mysqlCon.setXaStatus(TxState.TX_ROLLBACKED_STATE);
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -346,9 +326,9 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
session.setXaState(TxState.TX_INITIALIZE_STATE);
cleanAndFeedback();
}
break;
// 'xa rollback' err
case TX_PREPARED_STATE:
// 'xa rollback' err
} else if (mysqlCon.getXaStatus() == TxState.TX_PREPARED_STATE) {
mysqlCon.setXaStatus(TxState.TX_ROLLBACK_FAILED_STATE);
mysqlCon.quit();
XAStateLog.saveXARecoverylog(session.getSessionXaID(), mysqlCon);
@@ -356,18 +336,14 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
if (decrementCountBy(1)) {
cleanAndFeedback();
}
break;
default:
// LOGGER.error("Wrong XA status flag!");
} else {// LOGGER.error("Wrong XA status flag!");
}
}
}
private void cleanAndFeedback() {
switch (session.getXaState()) {
// rollbak success
case TX_INITIALIZE_STATE:
// clear all resources
if (session.getXaState() == TxState.TX_INITIALIZE_STATE) {// clear all resources
XAStateLog.saveXARecoverylog(session.getSessionXaID(), TxState.TX_ROLLBACKED_STATE);
//取消限制
session.cancelableStatusSet(NonBlockingSession.CANCEL_STATUS_INIT);
@@ -377,20 +353,18 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
return;
}
session.getSource().write(send);
break;
//partitionly commited,must commit again
case TX_ROLLBACK_FAILED_STATE:
case TX_PREPARED_STATE:
//partitionly commited,must commit again
} else if (session.getXaState() == TxState.TX_ROLLBACK_FAILED_STATE || session.getXaState() == TxState.TX_PREPARED_STATE) {
MySQLConnection errConn = session.releaseExcept(session.getXaState());
if (errConn != null) {
XAStateLog.saveXARecoverylog(session.getSessionXaID(), session.getXaState());
if (++try_rollback_times < ROLLBACK_TIMES) {
// 多试几次
rollback();
}
else {
} else {
StringBuffer closeReason = new StringBuffer("ROLLBCAK FAILED but it will try to ROLLBACK repeatedly in backend until it is success!");
if(error!= null){
if (error != null) {
closeReason.append(", the ERROR is ");
closeReason.append(error);
}
@@ -407,9 +381,9 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
session.getSource().write(toSend);
}
}
break;
// rollbak success,but closed coon must remove
default:
// rollbak success,but closed coon must remove
} else {
removeQuitConn();
XAStateLog.saveXARecoverylog(session.getSessionXaID(), TxState.TX_ROLLBACKED_STATE);
session.setXaState(TxState.TX_INITIALIZE_STATE);
@@ -418,7 +392,7 @@ public class XARollbackNodesHandler extends AbstractRollbackNodesHandler {
return;
}
session.getSource().write(sendData);
break;
}
}
private void removeQuitConn() {

View File

@@ -85,28 +85,27 @@ public class HandlerTool {
Item ret = null;
if (sel.basicConstItem())
return sel;
switch (sel.type()) {
case FUNC_ITEM:
case COND_ITEM:
Item.ItemType i = sel.type();
if (i == Item.ItemType.FUNC_ITEM || i == Item.ItemType.COND_ITEM) {
ItemFunc func = (ItemFunc) sel;
if (func.getPushDownName()==null ||func.getPushDownName().length()==0) {
if (func.getPushDownName() == null || func.getPushDownName().length() == 0) {
// 自己计算
ret = createFunctionItem(func, fields, startIndex, allPushDown, type, charset);
} else {
ret = createFieldItem(func, fields, startIndex);
}
break;
case SUM_FUNC_ITEM:
} else if (i == Item.ItemType.SUM_FUNC_ITEM) {
ItemSum sumFunc = (ItemSum) sel;
if (type != HandlerType.GROUPBY) {
ret = createFieldItem(sumFunc, fields, startIndex);
} else if (sumFunc.getPushDownName()==null ||sumFunc.getPushDownName().length()==0) {
} else if (sumFunc.getPushDownName() == null || sumFunc.getPushDownName().length() == 0) {
ret = createSumItem(sumFunc, fields, startIndex, allPushDown, type, charset);
} else {
ret = createPushDownGroupBy(sumFunc, fields, startIndex);
}
break;
default:
} else {
ret = createFieldItem(sel, fields, startIndex);
}
if (ret == null)

View File

@@ -68,19 +68,15 @@ public abstract class FilePath {
private static void registerDefaultProviders() {
if (providers == null || defaultProvider == null) {
Map<String, FilePath> map = Collections.synchronizedMap(new HashMap<String, FilePath>());
for (String c : new String[] { "com.actionsky.ares.partition.store.fs.FilePathDisk",
"com.actionsky.ares.partition.store.fs.FilePathNio",
"com.actionsky.ares.partition.store.fs.FilePathNioMapped" }) {
try {
FilePath p = (FilePath) Class.forName(c).newInstance();
map.put(p.getScheme(), p);
if (defaultProvider == null) {
defaultProvider = p;
}
} catch (Exception e) {
// ignore - the files may be excluded in purpose
}
FilePathDisk p = new FilePathDisk();
map.put(p.getScheme(), p);
if (defaultProvider == null) {
defaultProvider = p;
}
FilePathNio p2 = new FilePathNio();
map.put(p2.getScheme(), p2);
FilePathNioMapped p3 = new FilePathNioMapped();
map.put(p3.getScheme(), p3);
providers = map;
}
}

View File

@@ -19,6 +19,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.LockSupport;
import io.mycat.config.ErrorCode;
import io.mycat.util.ResourceUtil;
import io.mycat.util.exception.TmpFileException;
/**
@@ -277,9 +278,9 @@ public class FilePathDisk extends FilePath {
if (!fileName.startsWith("/")) {
fileName = "/" + fileName;
}
InputStream in = getClass().getResourceAsStream(fileName);
InputStream in = ResourceUtil.getResourceAsStream(fileName);
if (in == null) {
in = Thread.currentThread().getContextClassLoader().getResourceAsStream(fileName);
in = ResourceUtil.getResourceAsStreamForCurrentThread(fileName);
}
if (in == null) {
throw new FileNotFoundException("resource " + fileName);

View File

@@ -29,11 +29,11 @@ public class XARecoverCallback implements SQLQueryResultListener<SQLQueryResult<
}
public void onResult(SQLQueryResult<Map<String, String>> result) {
if (result.isSuccess()) {
LOGGER.debug("[CALLBACK][XA " + operator + "] when Mycat start");
LOGGER.debug("[CALLBACK][XA " + operator + "] when server start");
XAStateLog.updateXARecoverylog(logEntry.getCoordinatorId(), logEntry.getHost(), logEntry.getPort(), logEntry.getSchema(), txState);
XAStateLog.writeCheckpoint(logEntry.getCoordinatorId());
} else {
LOGGER.warn("[CALLBACK][XA " + operator + "] when Mycat start,but failed");
LOGGER.warn("[CALLBACK][XA " + operator + "] when server start,but failed");
}
}
}

View File

@@ -48,8 +48,8 @@ public class FileSystemRepository implements Repository{
// ConfigProperties configProperties = Configuration.getConfigProperties();
// String baseDir = configProperties.getLogBaseDir();
// String baseName = configProperties.getLogBaseName();
MycatConfig mycatconfig = MycatServer.getInstance().getConfig();
SystemConfig systemConfig = mycatconfig.getSystem();
MycatConfig config = MycatServer.getInstance().getConfig();
SystemConfig systemConfig = config.getSystem();
String baseDir =systemConfig.getXARecoveryLogBaseDir();
String baseName = systemConfig.getXARecoveryLogBaseName();

View File

@@ -23,6 +23,10 @@
*/
package io.mycat.cache;
import io.mycat.cache.impl.EnchachePooFactory;
import io.mycat.cache.impl.LevelDBCachePooFactory;
import io.mycat.cache.impl.MapDBCachePooFactory;
import io.mycat.util.ResourceUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -64,8 +68,7 @@ public class CacheService {
private void init(boolean isLowerCaseTableNames) throws Exception {
Properties props = new Properties();
props.load(CacheService.class
.getResourceAsStream("/cacheservice.properties"));
props.load(ResourceUtil.getResourceAsStream("/cacheservice.properties"));
final String poolFactoryPref = "factory.";
final String poolKeyPref = "pool.";
final String layedPoolKeyPref = "layedpool.";
@@ -143,9 +146,23 @@ public class CacheService {
private void createPoolFactory(String factryType, String factryClassName)
throws Exception {
CachePoolFactory factry = (CachePoolFactory) Class.forName(
factryClassName).newInstance();
poolFactorys.put(factryType, factry);
String lowerClass = factryClassName.toLowerCase();
switch(lowerClass){
case "ehcache":
poolFactorys.put(factryType, new EnchachePooFactory());
break;
case "leveldb":
poolFactorys.put(factryType, new LevelDBCachePooFactory());
break;
case "mapdb":
poolFactorys.put(factryType, new MapDBCachePooFactory());
break;
default:
CachePoolFactory factry = (CachePoolFactory) Class.forName(
factryClassName).newInstance();
poolFactorys.put(factryType, factry);
}
}

View File

@@ -30,7 +30,7 @@ package io.mycat.config;
*/
public interface Alarms {
/** 默认报警关键词 **/
public static final String DEFAULT = "#!MyCat#";
public static final String DEFAULT = "#!Server#";
/** 集群无有效的节点可提供服务 **/
public static final String CLUSTER_EMPTY = "#!CLUSTER_EMPTY#";

View File

@@ -59,7 +59,6 @@ public class ConfigInitializer {
private static final Logger LOGGER = Logger.getLogger( ConfigInitializer.class );
private volatile SystemConfig system;
private volatile MycatCluster cluster;
private volatile FirewallConfig firewall;
private volatile Map<String, UserConfig> users;
private volatile Map<String, SchemaConfig> schemas;
@@ -85,7 +84,6 @@ public class ConfigInitializer {
//权限管理
this.firewall = configLoader.getFirewallConfig();
this.cluster = initCobarCluster(configLoader);
//不同类型的全局序列处理器的配置加载
if (system.getSequnceHandlerType() == SystemConfig.SEQUENCEHANDLER_MYSQLDB) {
@@ -206,9 +204,6 @@ public class ConfigInitializer {
return system;
}
public MycatCluster getCluster() {
return cluster;
}
public FirewallConfig getFirewall() {
return firewall;
@@ -233,10 +228,6 @@ public class ConfigInitializer {
public Map<ERTable, Set<ERTable>> getErRelations() {
return erRelations;
}
private MycatCluster initCobarCluster(XMLConfigLoader configLoader) {
return new MycatCluster(configLoader.getClusterConfig());
}
private Map<String, PhysicalDBPool> initDataHosts(SchemaLoader schemaLoader) {
Map<String, DataHostConfig> nodeConfs = schemaLoader.getDataHosts();
//根据DataHost建立PhysicalDBPool其实就是实际数据库连接池每个DataHost对应一个PhysicalDBPool

View File

@@ -1,59 +0,0 @@
/*
* Copyright (c) 2013, OpenCloudDB/MyCAT and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software;Designed and Developed mainly by many Chinese
* opensource volunteers. you can redistribute it and/or modify it under the
* terms of the GNU General Public License version 2 only, as published by the
* Free Software Foundation.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Any questions about this component can be directed to it's project Web address
* https://code.google.com/p/opencloudb/.
*
*/
package io.mycat.config;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import io.mycat.config.model.ClusterConfig;
import io.mycat.config.model.MycatNodeConfig;
/**
* @author mycat
*/
public final class MycatCluster {
private final Map<String, MycatNode> nodes;
private final Map<String, List<String>> groups;
public MycatCluster(ClusterConfig clusterConf) {
this.nodes = new HashMap<String, MycatNode>(clusterConf.getNodes().size());
this.groups = clusterConf.getGroups();
for (MycatNodeConfig conf : clusterConf.getNodes().values()) {
String name = conf.getName();
MycatNode node = new MycatNode(conf);
this.nodes.put(name, node);
}
}
public Map<String, MycatNode> getNodes() {
return nodes;
}
public Map<String, List<String>> getGroups() {
return groups;
}
}

View File

@@ -56,8 +56,6 @@ public class MycatConfig {
private static final int RELOAD_ALL = 3;
private volatile SystemConfig system;
private volatile MycatCluster cluster;
private volatile MycatCluster _cluster;
private volatile FirewallConfig firewall;
private volatile FirewallConfig _firewall;
private volatile Map<String, UserConfig> users;
@@ -90,7 +88,6 @@ public class MycatConfig {
}
this.firewall = confInit.getFirewall();
this.cluster = confInit.getCluster();
//初始化重加载配置时间
this.reloadTime = TimeUtil.currentTimeMillis();
@@ -185,14 +182,6 @@ public class MycatConfig {
return _erRelations;
}
public MycatCluster getCluster() {
return cluster;
}
public MycatCluster getBackupCluster() {
return _cluster;
}
public FirewallConfig getFirewall() {
return firewall;
}
@@ -215,16 +204,16 @@ public class MycatConfig {
public void reload(Map<String, UserConfig> newUsers, Map<String, SchemaConfig> newSchemas,
Map<String, PhysicalDBNode> newDataNodes, Map<String, PhysicalDBPool> newDataHosts,
Map<ERTable, Set<ERTable>> newErRelations, MycatCluster newCluster, FirewallConfig newFirewall,
Map<ERTable, Set<ERTable>> newErRelations, FirewallConfig newFirewall,
boolean reloadAll) {
apply(newUsers, newSchemas, newDataNodes, newDataHosts, newErRelations, newCluster, newFirewall, reloadAll);
apply(newUsers, newSchemas, newDataNodes, newDataHosts, newErRelations, newFirewall, reloadAll);
this.reloadTime = TimeUtil.currentTimeMillis();
this.status = reloadAll ? RELOAD_ALL : RELOAD;
}
public boolean canRollback() {
if (_users == null || _schemas == null || _dataNodes == null || _dataHosts == null || _cluster == null
if (_users == null || _schemas == null || _dataNodes == null || _dataHosts == null
|| _firewall == null || status == ROLLBACK) {
return false;
} else {
@@ -234,9 +223,9 @@ public class MycatConfig {
public void rollback(Map<String, UserConfig> users, Map<String, SchemaConfig> schemas,
Map<String, PhysicalDBNode> dataNodes, Map<String, PhysicalDBPool> dataHosts,
Map<ERTable, Set<ERTable>> erRelations, MycatCluster cluster, FirewallConfig firewall) {
Map<ERTable, Set<ERTable>> erRelations, FirewallConfig firewall) {
apply(users, schemas, dataNodes, dataHosts, erRelations, cluster, firewall, status==RELOAD_ALL);
apply(users, schemas, dataNodes, dataHosts, erRelations, firewall, status==RELOAD_ALL);
this.rollbackTime = TimeUtil.currentTimeMillis();
this.status = ROLLBACK;
}
@@ -339,7 +328,6 @@ public class MycatConfig {
Map<String, PhysicalDBNode> newDataNodes,
Map<String, PhysicalDBPool> newDataHosts,
Map<ERTable, Set<ERTable>> newErRelations,
MycatCluster newCluster,
FirewallConfig newFirewall,
boolean isLoadAll) {
final ReentrantLock lock = this.lock;
@@ -364,7 +352,6 @@ public class MycatConfig {
this._users = this.users;
this._schemas = this.schemas;
this._cluster = this.cluster;
this._firewall = this.firewall;
this._erRelations = this.erRelations ;
// comment BY huqing.yan and will reopen later
@@ -389,7 +376,6 @@ public class MycatConfig {
}
this.users = newUsers;
this.schemas = newSchemas;
this.cluster = newCluster;
this.firewall = newFirewall;
this.erRelations = newErRelations;
} finally {

View File

@@ -130,8 +130,8 @@ public class MycatPrivileges implements FrontendPrivileges {
@Override
public boolean checkFirewallWhiteHostPolicy(String user, String host) {
MycatConfig mycatConfig = MycatServer.getInstance().getConfig();
FirewallConfig firewallConfig = mycatConfig.getFirewall();
MycatConfig config = MycatServer.getInstance().getConfig();
FirewallConfig firewallConfig = config.getFirewall();
if (!checkManagerPrivilege(user)) {
// return and don't trigger firewall alarm
@@ -144,7 +144,7 @@ public class MycatPrivileges implements FrontendPrivileges {
Map<String, List<UserConfig>> whitehost = firewallConfig.getWhitehost();
if (whitehost == null || whitehost.size() == 0) {
Map<String, UserConfig> users = mycatConfig.getUsers();
Map<String, UserConfig> users = config.getUsers();
isPassed = users.containsKey(user);
} else {
@@ -217,19 +217,18 @@ public class MycatPrivileges implements FrontendPrivileges {
return true;
}
int index = -1;
switch (chekctype) {
case INSERT:
if (chekctype == Checktype.INSERT) {
index = 0;
break;
case UPDATE:
} else if (chekctype == Checktype.UPDATE) {
index = 1;
break;
case SELECT:
} else if (chekctype == Checktype.SELECT) {
index = 2;
break;
case DELETE:
} else if (chekctype == Checktype.DELETE) {
index = 3;
break;
}
if (tablePrivilege.getDml()[index] > 0) {
return true;

View File

@@ -32,9 +32,10 @@ public abstract class Versions {
public static final byte PROTOCOL_VERSION = 10;
/**服务器版本**/
public static byte[] SERVER_VERSION = "5.6.29-mycat-2.17.04.0-20170508103825".getBytes();
public static byte[] VERSION_COMMENT = "MyCat Server (OpenCloundDB)".getBytes();
public static byte[] SERVER_VERSION = "5.6.29-mycat-2.17.06.0-20170703165824".getBytes();
public static byte[] VERSION_COMMENT = "Mycat Server".getBytes();
public static String ANNOTATION_NAME = "mycat:";
public static final String ROOT_PREFIX = "mycat";
public static void setServerVersion(String version) {
byte[] mysqlVersionPart = version.getBytes();
@@ -45,10 +46,10 @@ public abstract class Versions {
}
// 重新拼接mycat version字节数组
byte[] newMycatVersion = new byte[mysqlVersionPart.length + SERVER_VERSION.length - startIndex];
System.arraycopy(mysqlVersionPart, 0, newMycatVersion, 0, mysqlVersionPart.length);
System.arraycopy(SERVER_VERSION, startIndex, newMycatVersion, mysqlVersionPart.length,
byte[] newVersion = new byte[mysqlVersionPart.length + SERVER_VERSION.length - startIndex];
System.arraycopy(mysqlVersionPart, 0, newVersion, 0, mysqlVersionPart.length);
System.arraycopy(SERVER_VERSION, startIndex, newVersion, mysqlVersionPart.length,
SERVER_VERSION.length - startIndex);
SERVER_VERSION = newMycatVersion;
SERVER_VERSION = newVersion;
}
}

View File

@@ -35,6 +35,7 @@ public abstract class Versions {
public static byte[] SERVER_VERSION = "@server-version@".getBytes();
public static byte[] VERSION_COMMENT = "@version-comment@".getBytes();
public static String ANNOTATION_NAME = "@annotation-name@";
public static final String ROOT_PREFIX = "@root_prefix@";
public static void setServerVersion(String version) {
byte[] mysqlVersionPart = version.getBytes();
@@ -45,10 +46,10 @@ public abstract class Versions {
}
// 重新拼接mycat version字节数组
byte[] newMycatVersion = new byte[mysqlVersionPart.length + SERVER_VERSION.length - startIndex];
System.arraycopy(mysqlVersionPart, 0, newMycatVersion, 0, mysqlVersionPart.length);
System.arraycopy(SERVER_VERSION, startIndex, newMycatVersion, mysqlVersionPart.length,
byte[] newVersion = new byte[mysqlVersionPart.length + SERVER_VERSION.length - startIndex];
System.arraycopy(mysqlVersionPart, 0, newVersion, 0, mysqlVersionPart.length);
System.arraycopy(SERVER_VERSION, startIndex, newVersion, mysqlVersionPart.length,
SERVER_VERSION.length - startIndex);
SERVER_VERSION = newMycatVersion;
SERVER_VERSION = newVersion;
}
}

View File

@@ -9,6 +9,7 @@ import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import io.mycat.config.Versions;
import org.slf4j.Logger; import org.slf4j.LoggerFactory;
/**
@@ -121,7 +122,7 @@ public class DynaClassLoader {
*/
public Class<?> loadClass(String name) throws ClassNotFoundException {
if (name.startsWith("java") || name.startsWith("sun")
|| name.startsWith("io.mycat")) {
|| name.startsWith("io."+ Versions.ROOT_PREFIX)) {
return super.loadClass(name);
}
DynaClass dynaClass = loadedDynaClassMap.get(name);

View File

@@ -1,5 +1,7 @@
package io.mycat.config.loader.console;
import io.mycat.config.Versions;
/**
* 专门用来操作zookeeper路径的文件信息
* 源文件名ZkPath.java
@@ -23,7 +25,7 @@ public enum ZookeeperPath {
* 最基础的mycat节点
* @字段说明 FLOW_ZK_PATH_ONLINE
*/
FLOW_ZK_PATH_BASE("mycat"),
FLOW_ZK_PATH_BASE(Versions.ROOT_PREFIX),
/**
* 在当前在线的节点

View File

@@ -25,7 +25,6 @@ package io.mycat.config.loader.xml;
import java.util.Map;
import io.mycat.config.model.ClusterConfig;
import io.mycat.config.model.FirewallConfig;
import io.mycat.config.model.SystemConfig;
import io.mycat.config.model.UserConfig;
@@ -39,28 +38,18 @@ public class XMLConfigLoader{
/** unmodifiable */
private final Map<String, UserConfig> users;
private final FirewallConfig firewall;
private final ClusterConfig cluster;
public XMLConfigLoader() {
XMLServerLoader serverLoader = new XMLServerLoader();
this.system = serverLoader.getSystem();
this.users = serverLoader.getUsers();
this.firewall = serverLoader.getFirewall();
this.cluster = serverLoader.getCluster();
}
public ClusterConfig getClusterConfig() {
return cluster;
}
public FirewallConfig getFirewallConfig() {
return firewall;
}
public UserConfig getUserConfig(String user) {
return users.get(user);
}
public Map<String, UserConfig> getUserConfigs() {
return users;
}

View File

@@ -31,6 +31,8 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import io.mycat.route.function.*;
import io.mycat.util.ResourceUtil;
import org.w3c.dom.Element;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
@@ -40,7 +42,6 @@ import io.mycat.config.model.rule.TableRuleConfig;
import io.mycat.config.util.ConfigException;
import io.mycat.config.util.ConfigUtil;
import io.mycat.config.util.ParameterMapping;
import io.mycat.route.function.AbstractPartitionAlgorithm;
import io.mycat.util.SplitUtil;
/**
@@ -80,8 +81,8 @@ public class XMLRuleLoader {
InputStream dtd = null;
InputStream xml = null;
try {
dtd = XMLRuleLoader.class.getResourceAsStream(dtdFile);
xml = XMLRuleLoader.class.getResourceAsStream(xmlFile);
dtd = ResourceUtil.getResourceAsStream(dtdFile);
xml = ResourceUtil.getResourceAsStream(xmlFile);
//读取出语意树
Element root = ConfigUtil.getDocument(dtd, xml)
.getDocumentElement();
@@ -217,13 +218,31 @@ public class XMLRuleLoader {
private AbstractPartitionAlgorithm createFunction(String name, String clazz)
throws ClassNotFoundException, InstantiationException,
IllegalAccessException, InvocationTargetException {
Class<?> clz = Class.forName(clazz);
//判断是否继承AbstractPartitionAlgorithm
if (!AbstractPartitionAlgorithm.class.isAssignableFrom(clz)) {
throw new IllegalArgumentException("rule function must implements "
+ AbstractPartitionAlgorithm.class.getName() + ", name=" + name);
String lowerClass = clazz.toLowerCase();
switch(lowerClass){
case "hash":
return new PartitionByLong();
case "stringhash":
return new PartitionByString();
case "enum":
return new PartitionByFileMap();
case "numberrange":
return new AutoPartitionByLong();
case "patternrange":
return new PartitionByPattern();
case "date":
return new PartitionByDate();
default:
Class<?> clz = Class.forName(clazz);
//判断是否继承AbstractPartitionAlgorithm
if (!AbstractPartitionAlgorithm.class.isAssignableFrom(clz)) {
throw new IllegalArgumentException("rule function must implements "
+ AbstractPartitionAlgorithm.class.getName() + ", name=" + name);
}
return (AbstractPartitionAlgorithm) clz.newInstance();
}
return (AbstractPartitionAlgorithm) clz.newInstance();
}
}

View File

@@ -37,6 +37,7 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import io.mycat.util.ResourceUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.w3c.dom.Element;
@@ -129,8 +130,8 @@ public class XMLSchemaLoader implements SchemaLoader {
InputStream dtd = null;
InputStream xml = null;
try {
dtd = XMLSchemaLoader.class.getResourceAsStream(dtdFile);
xml = XMLSchemaLoader.class.getResourceAsStream(xmlFile);
dtd = ResourceUtil.getResourceAsStream(dtdFile);
xml = ResourceUtil.getResourceAsStream(xmlFile);
Element root = ConfigUtil.getDocument(dtd, xml).getDocumentElement();
//先加载所有的DataHost
loadDataHosts(root);

View File

@@ -23,25 +23,8 @@
*/
package io.mycat.config.loader.xml;
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import org.w3c.dom.Element;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import com.alibaba.druid.wall.WallConfig;
import io.mycat.config.Versions;
import io.mycat.config.model.ClusterConfig;
import io.mycat.config.model.FirewallConfig;
import io.mycat.config.model.SystemConfig;
import io.mycat.config.model.UserConfig;
@@ -50,7 +33,16 @@ import io.mycat.config.util.ConfigException;
import io.mycat.config.util.ConfigUtil;
import io.mycat.config.util.ParameterMapping;
import io.mycat.util.DecryptUtil;
import io.mycat.util.ResourceUtil;
import io.mycat.util.SplitUtil;
import org.w3c.dom.Element;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.InvocationTargetException;
import java.util.*;
/**
* @author mycat
@@ -60,7 +52,6 @@ public class XMLServerLoader {
private final SystemConfig system;
private final Map<String, UserConfig> users;
private final FirewallConfig firewall;
private ClusterConfig cluster;
public XMLServerLoader() {
this.system = new SystemConfig();
@@ -81,17 +72,14 @@ public class XMLServerLoader {
return firewall;
}
public ClusterConfig getCluster() {
return cluster;
}
private void load() {
//读取server.xml配置
InputStream dtd = null;
InputStream xml = null;
try {
dtd = XMLServerLoader.class.getResourceAsStream("/server.dtd");
xml = XMLServerLoader.class.getResourceAsStream("/server.xml");
dtd = ResourceUtil.getResourceAsStream("/server.dtd");
xml = ResourceUtil.getResourceAsStream("/server.xml");
Element root = ConfigUtil.getDocument(dtd, xml).getDocumentElement();
//加载System标签
@@ -100,9 +88,6 @@ public class XMLServerLoader {
//加载User标签
loadUsers(root);
//加载集群配置
this.cluster = new ClusterConfig(root, system.getServerPort());
//加载全局SQL防火墙
loadFirewall(root);
} catch (ConfigException e) {
@@ -187,7 +172,7 @@ public class XMLServerLoader {
Map<String, Object> props = ConfigUtil.loadElements(e);
String password = (String) props.get("password");
String usingDecrypt = (String) props.get("usingDecrypt");
String passwordDecrypt = DecryptUtil.mycatDecrypt(usingDecrypt, name, password);
String passwordDecrypt = DecryptUtil.decrypt(usingDecrypt, name, password);
user.setName(name);
user.setPassword(passwordDecrypt);
user.setEncryptPassword(password);

View File

@@ -4,6 +4,7 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.Properties;
import io.mycat.util.ResourceUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -103,7 +104,7 @@ public class ZkConfig {
private static Properties LoadMyidPropersites() {
Properties pros = new Properties();
try (InputStream configIS = ZkConfig.class.getResourceAsStream(ZK_CONFIG_FILE_NAME)) {
try (InputStream configIS = ResourceUtil.getResourceAsStream(ZK_CONFIG_FILE_NAME)) {
if (configIS == null) {
return null;
}

View File

@@ -36,12 +36,6 @@ public enum ZkParamCfg {
* @字段说明 zk_CFG_MYID
*/
ZK_CFG_MYID("myid"),
MYCAT_SERVER_TYPE("type"),
MYCAT_BOOSTER_DATAHOSTS("boosterDataHosts"),
/**
* 集群中所有节点的名称信息
* @字段说明 ZK_CFG_CLUSTER_NODES

View File

@@ -7,11 +7,12 @@ import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import io.mycat.config.Versions;
import io.mycat.config.loader.zkprocess.entity.rule.function.Function;
import io.mycat.config.loader.zkprocess.entity.rule.tablerule.TableRule;
@XmlAccessorType(XmlAccessType.FIELD)
@XmlRootElement(namespace = "http://io.mycat/", name = "rule")
@XmlRootElement(namespace = "http://io."+ Versions.ROOT_PREFIX+"/", name = "rule")
public class Rules {
/**

View File

@@ -7,12 +7,13 @@ import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlRootElement;
import io.mycat.config.Versions;
import io.mycat.config.loader.zkprocess.entity.schema.datahost.DataHost;
import io.mycat.config.loader.zkprocess.entity.schema.datanode.DataNode;
import io.mycat.config.loader.zkprocess.entity.schema.schema.Schema;
@XmlAccessorType(XmlAccessType.FIELD)
@XmlRootElement(namespace = "http://io.mycat/", name = "schema")
@XmlRootElement(namespace = "http://io."+ Versions.ROOT_PREFIX+"/", name = "schema")
public class Schemas {
/**
* 配制的逻辑表信息

View File

@@ -7,11 +7,12 @@ import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement;
import io.mycat.config.Versions;
import io.mycat.config.loader.zkprocess.entity.server.System;
import io.mycat.config.loader.zkprocess.entity.server.user.User;
@XmlAccessorType(XmlAccessType.FIELD)
@XmlRootElement(namespace = "http://io.mycat/", name = "server")
@XmlRootElement(namespace = "http://io."+ Versions.ROOT_PREFIX+"/", name = "server")
public class Server {
@XmlElement(required = true)

View File

@@ -1,6 +1,8 @@
@XmlSchema(xmlns = @XmlNs(prefix = "mycat", namespaceURI = "http://io.mycat/") , elementFormDefault = javax.xml.bind.annotation.XmlNsForm.QUALIFIED)
@XmlSchema(xmlns = @XmlNs(prefix = Versions.ROOT_PREFIX, namespaceURI = "http://io."+ Versions.ROOT_PREFIX+"/") , elementFormDefault = javax.xml.bind.annotation.XmlNsForm.QUALIFIED)
package io.mycat.config.loader.zkprocess.entity;
import io.mycat.config.Versions;
import javax.xml.bind.annotation.XmlNs;
import javax.xml.bind.annotation.XmlSchema;

View File

@@ -22,6 +22,8 @@ import javax.xml.stream.XMLStreamException;
import javax.xml.stream.XMLStreamReader;
import javax.xml.transform.stream.StreamSource;
import io.mycat.config.Versions;
import io.mycat.util.ResourceUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -114,7 +116,7 @@ public class XmlProcessBase {
if (null != name) {
marshaller.setProperty("com.sun.xml.internal.bind.xmlHeaders",
String.format("<!DOCTYPE mycat:%1$s SYSTEM \"%1$s.dtd\">", name));
String.format("<!DOCTYPE "+ Versions.ROOT_PREFIX+":%1$s SYSTEM \"%1$s.dtd\">", name));
}
Path path = Paths.get(inputPath);
@@ -149,7 +151,7 @@ public class XmlProcessBase {
if (null != name) {
marshaller.setProperty("com.sun.xml.internal.bind.xmlHeaders",
String.format("<!DOCTYPE mycat:%1$s SYSTEM \"%1$s.dtd\">", name));
String.format("<!DOCTYPE "+Versions.ROOT_PREFIX+":%1$s SYSTEM \"%1$s.dtd\">", name));
}
if (null != map && !map.isEmpty()) {
@@ -184,7 +186,7 @@ public class XmlProcessBase {
*/
public Object baseParseXmlToBean(String fileName) throws JAXBException, XMLStreamException {
// 搜索当前转化的文件
InputStream inputStream = XmlProcessBase.class.getResourceAsStream(fileName);
InputStream inputStream = ResourceUtil.getResourceAsStream(fileName);
// 如果能够搜索到文件
if (inputStream != null) {

View File

@@ -5,6 +5,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
import java.io.IOException;
import java.io.InputStream;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -139,7 +140,7 @@ public class EcachesxmlTozkLoader extends ZkMultLoader implements NotifyService
String path = ZookeeperPath.ZK_LOCAL_CFG_PATH.getKey() + name;
// 加载数据
InputStream input = EcachesxmlTozkLoader.class.getResourceAsStream(path);
InputStream input = ResourceUtil.getResourceAsStream(path);
checkNotNull(input, "read SeqFile file curr Path :" + path + " is null! must is not null");

View File

@@ -7,6 +7,7 @@ import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -175,7 +176,7 @@ public class RulesxmlTozkLoader extends ZkMultLoader implements NotifyService {
String path = ZookeeperPath.ZK_LOCAL_CFG_PATH.getKey() + name;
// 加载数据
InputStream input = RulesxmlTozkLoader.class.getResourceAsStream(path);
InputStream input = ResourceUtil.getResourceAsStream(path);
checkNotNull(input, "read Map file curr Path :" + path + " is null! must is not null");

View File

@@ -3,6 +3,7 @@ package io.mycat.config.loader.zkprocess.xmltozk.listen;
import java.io.IOException;
import java.io.InputStream;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -168,7 +169,7 @@ public class SequenceTozkLoader extends ZkMultLoader implements NotifyService {
String path = ZookeeperPath.ZK_LOCAL_CFG_PATH.getKey() + name;
// 加载数据
InputStream input = SequenceTozkLoader.class.getResourceAsStream(path);
InputStream input = ResourceUtil.getResourceAsStream(path);
if (null != input) {

View File

@@ -4,6 +4,7 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.List;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -194,7 +195,7 @@ public class ServerxmlTozkLoader extends ZkMultLoader implements NotifyService {
String path = ZookeeperPath.ZK_LOCAL_CFG_PATH.getKey() + name;
// 加载数据
InputStream input = SequenceTozkLoader.class.getResourceAsStream(path);
InputStream input = ResourceUtil.getResourceAsStream(path);
if (null != input) {

View File

@@ -59,34 +59,33 @@ public class BinlogPauseStatusListener extends ZkMultLoader implements NotifySe
}
String binlogPause = basePath + ShowBinlogStatus.BINLOG_PAUSE_INSTANCES;
String instancePath = ZKPaths.makePath(binlogPause, ZkConfig.getInstance().getValue(ZkParamCfg.ZK_CFG_MYID));
switch (pauseInfo.getStatus()) {
case ON:
MycatServer.getInstance().getBackupLocked().compareAndSet(false, true);
if (ShowBinlogStatus.waitAllSession()) {
try {
ZKUtils.createTempNode(binlogPause, ZkConfig.getInstance().getValue(ZkParamCfg.ZK_CFG_MYID));
} catch (Exception e) {
LOGGER.warn("create binlogPause instance failed", e);
}
}
break;
case TIMEOUT:
ShowBinlogStatus.setWaiting(false);
break;
case OFF:
while(ShowBinlogStatus.isWaiting()){
LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(1000));
}
if (pauseInfo.getStatus() == BinlogPause.BinlogPauseStatus.ON) {
MycatServer.getInstance().getBackupLocked().compareAndSet(false, true);
if (ShowBinlogStatus.waitAllSession()) {
try {
if (this.getCurator().checkExists().forPath(instancePath) != null) {
this.getCurator().delete().forPath(instancePath);
}
ZKUtils.createTempNode(binlogPause, ZkConfig.getInstance().getValue(ZkParamCfg.ZK_CFG_MYID));
} catch (Exception e) {
LOGGER.warn("delete binlogPause instance failed", e);
} finally {
MycatServer.getInstance().getBackupLocked().compareAndSet(true, false);
LOGGER.warn("create binlogPause instance failed", e);
}
break;
}
} else if (pauseInfo.getStatus() == BinlogPause.BinlogPauseStatus.TIMEOUT) {
ShowBinlogStatus.setWaiting(false);
} else if (pauseInfo.getStatus() == BinlogPause.BinlogPauseStatus.OFF) {
while (ShowBinlogStatus.isWaiting()) {
LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(1000));
}
try {
if (this.getCurator().checkExists().forPath(instancePath) != null) {
this.getCurator().delete().forPath(instancePath);
}
} catch (Exception e) {
LOGGER.warn("delete binlogPause instance failed", e);
} finally {
MycatServer.getInstance().getBackupLocked().compareAndSet(true, false);
}
}

View File

@@ -7,6 +7,7 @@ import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -134,8 +135,7 @@ public class EcacheszkToxmlLoader extends ZkMultLoader implements NotifyService
Ehcache ehcache = parseJsonEhcacheService.parseJsonToBean(ehcacheZkDirectory.getDataValue());
String outputPath = EcacheszkToxmlLoader.class.getClassLoader()
.getResource(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey()).getPath();
String outputPath = ResourceUtil.getResourcePathFromRoot(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey());
outputPath=new File(outputPath).getPath()+File.separator;
outputPath += EHCACHE_NAME;
@@ -172,8 +172,7 @@ public class EcacheszkToxmlLoader extends ZkMultLoader implements NotifyService
private void writeCacheservice(String name, String value) {
// 加载数据
String path = RuleszkToxmlLoader.class.getClassLoader().getResource(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey())
.getPath();
String path = ResourceUtil.getResourcePathFromRoot(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey());
checkNotNull(path, "write ecache file curr Path :" + path + " is null! must is not null");
path=new File(path).getPath()+File.separator;

View File

@@ -11,6 +11,7 @@ import java.util.List;
import io.mycat.MycatServer;
import io.mycat.manager.response.ReloadConfig;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -129,8 +130,7 @@ public class RuleszkToxmlLoader extends ZkMultLoader implements NotifyService {
LOGGER.info("RuleszkToxmlLoader notifyProcess write mapFile is success ");
// 数配制信息写入文件
String path = RuleszkToxmlLoader.class.getClassLoader().getResource(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey())
.getPath();
String path = ResourceUtil.getResourcePathFromRoot(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey());
path=new File(path).getPath()+File.separator;
path =path+WRITEPATH;
@@ -244,8 +244,7 @@ public class RuleszkToxmlLoader extends ZkMultLoader implements NotifyService {
private void writeMapFile(String name, String value) {
// 加载数据
String path = RuleszkToxmlLoader.class.getClassLoader().getResource(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey())
.getPath();
String path = ResourceUtil.getResourcePathFromRoot(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey());
checkNotNull(path, "write Map file curr Path :" + path + " is null! must is not null");
path=new File(path).getPath()+File.separator;

View File

@@ -5,6 +5,7 @@ import java.util.List;
import io.mycat.MycatServer;
import io.mycat.manager.response.ReloadConfig;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -122,8 +123,7 @@ public class SchemaszkToxmlLoader extends ZkMultLoader implements NotifyService
LOGGER.info("SchemasLoader notifyProcess zk to object zk schema Object :" + schema);
String path = SchemaszkToxmlLoader.class.getClassLoader()
.getResource(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey()).getPath();
String path = ResourceUtil.getResourcePathFromRoot(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey());
path=new File(path).getPath()+File.separator;
path += WRITEPATH;

View File

@@ -9,6 +9,7 @@ import java.io.IOException;
import io.mycat.MycatServer;
import io.mycat.manager.response.ReloadConfig;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -268,8 +269,7 @@ public class SequenceTopropertiesLoader extends ZkMultLoader implements NotifySe
private void writeMapFile(String name, String value) {
// 加载数据
String path = RuleszkToxmlLoader.class.getClassLoader().getResource(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey())
.getPath();
String path = ResourceUtil.getResourcePathFromRoot(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey());
checkNotNull(path, "write Map file curr Path :" + path + " is null! must is not null");

View File

@@ -11,6 +11,7 @@ import java.util.List;
import io.mycat.MycatServer;
import io.mycat.config.loader.zkprocess.comm.ZkConfig;
import io.mycat.manager.response.ReloadConfig;
import io.mycat.util.ResourceUtil;
import org.apache.curator.framework.CuratorFramework;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -138,8 +139,7 @@ public class ServerzkToxmlLoader extends ZkMultLoader implements NotifyService {
LOGGER.info("ServerzkToxmlLoader notifyProcess zk to object zk server Object :" + server);
// 数配制信息写入文件
String path = ServerzkToxmlLoader.class.getClassLoader().getResource(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey())
.getPath();
String path = ResourceUtil.getResourcePathFromRoot(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey());
path=new File(path).getPath()+File.separator;
path += WRITEPATH;
@@ -248,8 +248,7 @@ public class ServerzkToxmlLoader extends ZkMultLoader implements NotifyService {
private void writeProperties(String name, String value) {
// 加载数据
String path = RuleszkToxmlLoader.class.getClassLoader().getResource(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey())
.getPath();
String path = ResourceUtil.getResourcePathFromRoot(ZookeeperPath.ZK_LOCAL_WRITE_PATH.getKey());
checkNotNull(path, "write properties curr Path :" + path + " is null! must is not null");

View File

@@ -1,141 +0,0 @@
/*
* Copyright (c) 2013, OpenCloudDB/MyCAT and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software;Designed and Developed mainly by many Chinese
* opensource volunteers. you can redistribute it and/or modify it under the
* terms of the GNU General Public License version 2 only, as published by the
* Free Software Foundation.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Any questions about this component can be directed to it's project Web address
* https://code.google.com/p/opencloudb/.
*
*/
package io.mycat.config.model;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.w3c.dom.Element;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import io.mycat.config.util.ConfigException;
import io.mycat.config.util.ConfigUtil;
import io.mycat.util.SplitUtil;
/**
* @author mycat
*/
public class ClusterConfig {
private final Map<String, MycatNodeConfig> nodes;
private final Map<String, List<String>> groups;
public ClusterConfig(Element root, int port) {
nodes = Collections.unmodifiableMap(loadNode(root, port));
groups = Collections.unmodifiableMap(loadGroup(root, nodes));
}
public Map<String, MycatNodeConfig> getNodes() {
return nodes;
}
public Map<String, List<String>> getGroups() {
return groups;
}
private static Map<String, MycatNodeConfig> loadNode(Element root, int port) {
Map<String, MycatNodeConfig> nodes = new HashMap<String, MycatNodeConfig>();
NodeList list = root.getElementsByTagName("node");
Set<String> hostSet = new HashSet<String>();
for (int i = 0, n = list.getLength(); i < n; i++) {
Node node = list.item(i);
if (node instanceof Element) {
Element element = (Element) node;
String name = element.getAttribute("name").trim();
if (nodes.containsKey(name)) {
throw new ConfigException("node name duplicated :" + name);
}
Map<String, Object> props = ConfigUtil.loadElements(element);
String host = (String) props.get("host");
if (null == host || "".equals(host)) {
throw new ConfigException("host empty in node: " + name);
}
if (hostSet.contains(host)) {
throw new ConfigException("node host duplicated :" + host);
}
String wei = (String) props.get("weight");
if (null == wei || "".equals(wei)) {
throw new ConfigException("weight should not be null in host:" + host);
}
int weight = Integer.parseInt(wei);
if (weight <= 0) {
throw new ConfigException("weight should be > 0 in host:" + host + " weight:" + weight);
}
MycatNodeConfig conf = new MycatNodeConfig(name, host, port, weight);
nodes.put(name, conf);
hostSet.add(host);
}
}
return nodes;
}
private static Map<String, List<String>> loadGroup(Element root, Map<String, MycatNodeConfig> nodes) {
Map<String, List<String>> groups = new HashMap<String, List<String>>();
NodeList list = root.getElementsByTagName("group");
for (int i = 0, n = list.getLength(); i < n; i++) {
Node node = list.item(i);
if (node instanceof Element) {
Element e = (Element) node;
String groupName = e.getAttribute("name").trim();
if (groups.containsKey(groupName)) {
throw new ConfigException("group duplicated : " + groupName);
}
Map<String, Object> props = ConfigUtil.loadElements(e);
String value = (String) props.get("nodeList");
if (null == value || "".equals(value)) {
throw new ConfigException("group should contain 'nodeList'");
}
String[] sList = SplitUtil.split(value, ',', true);
if (null == sList || sList.length == 0) {
throw new ConfigException("group should contain 'nodeList'");
}
for (String s : sList) {
if (!nodes.containsKey(s)) {
throw new ConfigException("[ node :" + s + "] in [ group:" + groupName + "] doesn't exist!");
}
}
List<String> nodeList = Arrays.asList(sList);
groups.put(groupName, nodeList);
}
}
if (!groups.containsKey("default")) {
List<String> nodeList = new ArrayList<String>(nodes.keySet());
groups.put("default", nodeList);
}
return groups;
}
}

View File

@@ -35,6 +35,7 @@ import javax.xml.transform.TransformerFactory;
import javax.xml.transform.dom.DOMSource;
import javax.xml.transform.stream.StreamResult;
import io.mycat.util.ResourceUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.w3c.dom.Document;
@@ -200,7 +201,7 @@ public final class FirewallConfig {
public InputSource resolveEntity(java.lang.String publicId, java.lang.String systemId) throws SAXException, java.io.IOException{
if (systemId.contains("server.dtd")){
//InputSource is = new InputSource(new ByteArrayInputStream("<?xml version=\"1.0\" encoding=\"UTF-8\"?>".getBytes()));
InputStream dtd = XMLServerLoader.class.getResourceAsStream("/server.dtd");
InputStream dtd = ResourceUtil.getResourceAsStream("/server.dtd");
InputSource is = new InputSource(dtd);
return is;
} else {
@@ -208,13 +209,4 @@ public final class FirewallConfig {
}
}
}
// public static void main(String[] args) throws Exception {
// List<UserConfig> userConfigs = new ArrayList<UserConfig>();
// UserConfig user = new UserConfig();
// user.setName("mycat");
// userConfigs.add(user);
// updateToFile("127.0.0.1",userConfigs);
// }
}

View File

@@ -68,7 +68,7 @@ public final class SystemConfig {
private static final boolean DEFAULT_USE_ZK_SWITCH = true;
private static final boolean DEFAULT_LOWER_CASE = true;
private static final String DEFAULT_TRANSACTION_BASE_DIR = "txlogs";
private static final String DEFAULT_TRANSACTION_BASE_NAME = "mycat-tx";
private static final String DEFAULT_TRANSACTION_BASE_NAME = "server-tx";
private static final int DEFAULT_TRANSACTION_ROTATE_SIZE = 16;
private static final long CHECKTABLECONSISTENCYPERIOD = 30 * 60 * 1000;
// 全局表一致性检测任务默认24小时调度一次
@@ -128,7 +128,7 @@ public final class SystemConfig {
private int usingAIO = 0;
private int packetHeaderSize = 4;
private int maxPacketSize = 16 * 1024 * 1024;
private int mycatNodeId=1;
private int serverNodeId =1;
private int useCompression =0;
private int useSqlStat = 1;
@@ -742,12 +742,12 @@ public final class SystemConfig {
this.usingAIO = usingAIO;
}
public int getMycatNodeId() {
return mycatNodeId;
public int getServerNodeId() {
return serverNodeId;
}
public void setMycatNodeId(int mycatNodeId) {
this.mycatNodeId = mycatNodeId;
public void setServerNodeId(int serverNodeId) {
this.serverNodeId = serverNodeId;
}
@Override
@@ -791,7 +791,7 @@ public final class SystemConfig {
+ ", usingAIO=" + usingAIO
+ ", packetHeaderSize=" + packetHeaderSize
+ ", maxPacketSize=" + maxPacketSize
+ ", mycatNodeId=" + mycatNodeId
+ ", serverNodeId=" + serverNodeId
+ "]";
}

View File

@@ -28,8 +28,8 @@ public class TxnLogProcessor extends Thread {
public TxnLogProcessor(BufferPool bufferPool) {
this.dateFormat = new ISO8601DateFormat();
this.queue = new LinkedBlockingQueue<TxnBinaryLog>(256);
MycatConfig mycatconfig = MycatServer.getInstance().getConfig();
SystemConfig systemConfig = mycatconfig.getSystem();
MycatConfig config = MycatServer.getInstance().getConfig();
SystemConfig systemConfig = config.getSystem();
this.store = new DailyRotateLogStore(systemConfig.getTransactionLogBaseDir(), systemConfig.getTransactionLogBaseName(),"log",systemConfig.getTransactionRatateSize());
}

View File

@@ -64,8 +64,8 @@ public class ManagerPrivileges extends MycatPrivileges {
}
protected boolean checkManagerPrivilege(String user) {
MycatConfig mycatConfig = MycatServer.getInstance().getConfig();
UserConfig rUser = mycatConfig.getUsers().get(user);
MycatConfig config = MycatServer.getInstance().getConfig();
UserConfig rUser = config.getUsers().get(user);
// Manager privilege must be assign explicitly
if (rUser == null || rUser.isManager() == false)
return false;

View File

@@ -39,6 +39,7 @@ import java.util.Date;
import javax.xml.parsers.ParserConfigurationException;
import io.mycat.util.ResourceUtil;
import org.slf4j.Logger; import org.slf4j.LoggerFactory;
import org.xml.sax.SAXException;
@@ -134,20 +135,20 @@ public final class ConfFileHandler {
InputStream dtdStream = new ByteArrayInputStream(new byte[0]);
File confDir = new File(SystemConfig.getHomePath(), "conf");
if (xmlFileName.equals("schema.xml")) {
dtdStream = MycatServer.class.getResourceAsStream("/schema.dtd");
dtdStream = ResourceUtil.getResourceAsStream("/schema.dtd");
if (dtdStream == null) {
dtdStream = new ByteArrayInputStream(readFileByBytes(new File(
confDir, "schema.dtd")));
}
} else if (xmlFileName.equals("server.xml")) {
dtdStream = MycatServer.class.getResourceAsStream("/server.dtd");
dtdStream = ResourceUtil.getResourceAsStream("/server.dtd");
if (dtdStream == null) {
dtdStream = new ByteArrayInputStream(readFileByBytes(new File(
confDir, "server.dtd")));
}
} else if (xmlFileName.equals("rule.xml")) {
dtdStream = MycatServer.class.getResourceAsStream("/rule.dtd");
dtdStream = ResourceUtil.getResourceAsStream("/rule.dtd");
if (dtdStream == null) {
dtdStream = new ByteArrayInputStream(readFileByBytes(new File(
confDir, "rule.dtd")));

View File

@@ -55,7 +55,7 @@ public final class ShowServerLog {
.getHeader(FIELD_COUNT);
private static final FieldPacket[] fields = new FieldPacket[FIELD_COUNT];
private static final EOFPacket eof = new EOFPacket();
private static final String DEFAULT_LOGFILE = "mycat.log";
public static final String DEFAULT_LOGFILE = "mycat.log";
private static final Logger LOGGER = LoggerFactory
.getLogger(ShowServerLog.class);
static {

View File

@@ -45,7 +45,6 @@ import io.mycat.backend.datasource.PhysicalDatasource;
import io.mycat.backend.mysql.nio.MySQLConnection;
import io.mycat.config.ConfigInitializer;
import io.mycat.config.ErrorCode;
import io.mycat.config.MycatCluster;
import io.mycat.config.MycatConfig;
import io.mycat.config.model.ERTable;
import io.mycat.config.model.FirewallConfig;
@@ -101,7 +100,6 @@ public final class ReloadConfig {
Map<String, PhysicalDBNode> newDataNodes = loader.getDataNodes();
Map<String, PhysicalDBPool> newDataHosts = loader.getDataHosts();
Map<ERTable, Set<ERTable>> newErRelations = loader.getErRelations();
MycatCluster newCluster = loader.getCluster();
FirewallConfig newFirewall = loader.getFirewall();
/* 1.2、实际链路检测 */
@@ -154,7 +152,7 @@ public final class ReloadConfig {
if ( isReloadStatusOK ) {
/* 2.3、 在老的配置上,应用新的配置,开始准备承接任务 */
config.reload(newUsers, newSchemas, newDataNodes, newDataHosts, newErRelations, newCluster, newFirewall, true);
config.reload(newUsers, newSchemas, newDataNodes, newDataHosts, newErRelations, newFirewall, true);
/* 2.4、 处理旧的资源 */
LOGGER.warn("1. clear old backend connection(size): " + NIOProcessor.backends_old.size());
@@ -211,11 +209,10 @@ public final class ReloadConfig {
Map<String, PhysicalDBNode> dataNodes = loader.getDataNodes();
Map<String, PhysicalDBPool> dataHosts = loader.getDataHosts();
Map<ERTable, Set<ERTable>> erRelations = loader.getErRelations();
MycatCluster cluster = loader.getCluster();
FirewallConfig firewall = loader.getFirewall();
/* 2、在老的配置上 应用新的配置 */
MycatServer.getInstance().getConfig().reload(users, schemas, dataNodes, dataHosts, erRelations, cluster, firewall, false);
MycatServer.getInstance().getConfig().reload(users, schemas, dataNodes, dataHosts, erRelations, firewall, false);
/* 3、清理缓存 */
MycatServer.getInstance().getCacheService().clearCache();
MycatServer.getInstance().reloadMetaData();

View File

@@ -34,7 +34,6 @@ import io.mycat.MycatServer;
import io.mycat.backend.datasource.PhysicalDBNode;
import io.mycat.backend.datasource.PhysicalDBPool;
import io.mycat.config.ErrorCode;
import io.mycat.config.MycatCluster;
import io.mycat.config.MycatConfig;
import io.mycat.config.model.ERTable;
import io.mycat.config.model.FirewallConfig;
@@ -77,7 +76,6 @@ public final class RollbackConfig {
Map<String, SchemaConfig> schemas = conf.getBackupSchemas();
Map<String, PhysicalDBNode> dataNodes = conf.getBackupDataNodes();
Map<String, PhysicalDBPool> dataHosts = conf.getBackupDataHosts();
MycatCluster cluster = conf.getBackupCluster();
FirewallConfig firewall = conf.getBackupFirewall();
Map<ERTable, Set<ERTable>> erRelations = conf.getBackupErRelations();
@@ -106,7 +104,7 @@ public final class RollbackConfig {
}
// 应用回滚
conf.rollback(users, schemas, dataNodes, dataHosts, erRelations, cluster, firewall);
conf.rollback(users, schemas, dataNodes, dataHosts, erRelations, firewall);
// 处理旧的资源
for (PhysicalDBPool dn : cNodes.values()) {

View File

@@ -27,6 +27,7 @@ import java.nio.ByteBuffer;
import io.mycat.backend.mysql.PacketUtil;
import io.mycat.config.Fields;
import io.mycat.config.Versions;
import io.mycat.manager.ManagerConnection;
import io.mycat.net.mysql.EOFPacket;
import io.mycat.net.mysql.FieldPacket;
@@ -38,7 +39,6 @@ import io.mycat.net.mysql.RowDataPacket;
*/
public final class SelectVersionComment {
private static final byte[] VERSION_COMMENT = "MyCat Server (monitor)".getBytes();
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];
@@ -71,7 +71,7 @@ public final class SelectVersionComment {
// write rows
byte packetId = eof.packetId;
RowDataPacket row = new RowDataPacket(FIELD_COUNT);
row.add(VERSION_COMMENT);
row.add(Versions.VERSION_COMMENT);
row.packetId = ++packetId;
buffer = row.write(buffer, c,true);

View File

@@ -115,7 +115,7 @@ public class ShowDirectMemory {
if(useOffHeapForMerge == 1) {
ConcurrentMap<Long, Long> concurrentHashMap = MycatServer.getInstance().
getMyCatMemory().
getServerMemory().
getResultMergeMemoryManager().getDirectMemorUsage();
for (Long key : concurrentHashMap.keySet()) {
@@ -203,7 +203,7 @@ public class ShowDirectMemory {
* 结果集合并时总共消耗的DirectMemory内存
*/
ConcurrentMap<Long, Long> concurrentHashMap = MycatServer.getInstance().
getMyCatMemory().
getServerMemory().
getResultMergeMemoryManager().getDirectMemorUsage();
for (Map.Entry<Long, Long> entry : concurrentHashMap.entrySet()) {
usedforMerge += entry.getValue();

View File

@@ -110,7 +110,7 @@ public final class ShowHelp {
// show
helps.put("show @@time.current", "Report current timestamp");
helps.put("show @@time.startup", "Report startup timestamp");
helps.put("show @@version", "Report Mycat Server version");
helps.put("show @@version", "Report Server version");
helps.put("show @@server", "Report server status");
helps.put("show @@threadpool", "Report threadPool status");
helps.put("show @@database", "Report databases");
@@ -145,10 +145,10 @@ public final class ShowHelp {
helps.put("show @@slow where schema = ?", "Report schema slow sql");
helps.put("show @@slow where datanode = ?", "Report datanode slow sql");
helps.put("show @@sysparam", "Report system param");
helps.put("show @@syslog limit=?", "Report system mycat.log");
helps.put("show @@white", "show mycat white host ");
helps.put("show @@white.set=?,?", "set mycat white host,[ip,user]");
helps.put("show @@directmemory=1 or 2", "show mycat direct memory usage");
helps.put("show @@syslog limit=?", "Report system log");
helps.put("show @@white", "show server white host ");
helps.put("show @@white.set=?,?", "set server white host,[ip,user]");
helps.put("show @@directmemory=1 or 2", "show server direct memory usage");
// switch
helps.put("switch @@datasource name:index", "Switch dataSource");
@@ -176,8 +176,8 @@ public final class ShowHelp {
helps.put("reload @@sqlstat=close", "Close real-time sql stat analyzer");
// offline/online
helps.put("offline", "Change MyCat status to OFF");
helps.put("online", "Change MyCat status to ON");
helps.put("offline", "Change Server status to OFF");
helps.put("online", "Change Server status to ON");
// clear
helps.put("clear @@slow where schema = ?", "Clear slow sql by schema");

View File

@@ -13,6 +13,7 @@ import io.mycat.backend.mysql.PacketUtil;
import io.mycat.config.Fields;
import io.mycat.config.model.SystemConfig;
import io.mycat.manager.ManagerConnection;
import io.mycat.manager.handler.ShowServerLog;
import io.mycat.net.mysql.EOFPacket;
import io.mycat.net.mysql.FieldPacket;
import io.mycat.net.mysql.ResultSetHeaderPacket;
@@ -64,7 +65,7 @@ public class ShowSysLog {
// write rows
byte packetId = eof.packetId;
String filename = SystemConfig.getHomePath() + File.separator + "logs" + File.separator + "mycat.log";
String filename = SystemConfig.getHomePath() + File.separator + "logs" + File.separator + ShowServerLog.DEFAULT_LOGFILE;
String[] lines = getLinesByLogFile(filename, numLines);

View File

@@ -112,31 +112,31 @@ public class ShowSysParam {
"Mysql_charset",
"Mysql_txIsolation",
"Mysql_sqlExecuteTimeout",
"Mycat_processorCheckPeriod",
"Mycat_dataNodeIdleCheckPeriod",
"Mycat_dataNodeHeartbeatPeriod",
"Mycat_bindIp",
"Mycat_serverPort",
"Mycat_managerPort"};
"Server_processorCheckPeriod",
"Server_dataNodeIdleCheckPeriod",
"Server_dataNodeHeartbeatPeriod",
"Server_bindIp",
"Server_serverPort",
"Server_managerPort"};
private static final String[] PARAM_DESCRIPTION = {
"主要用于指定系统可用的线程数默认值为Runtime.getRuntime().availableProcessors()方法返回的值。主要影响processorBufferPool、processorBufferLocalPercent、processorExecutor属性。NIOProcessor的个数也是由这个属性定义的所以调优的时候可以适当的调高这个属性。",
"指定每次分配Socket Direct Buffer的大小默认是4096个字节。这个属性也影响buffer pool的长度。",
"指定bufferPool计算 比例值。由于每次执行NIO读、写操作都需要使用到buffer系统初始化的时候会建立一定长度的buffer池来加快读、写的效率减少建立buffer的时间",
"主要用于指定NIOProcessor上共享的businessExecutor固定线程池大小。mycat在需要处理一些异步逻辑的时候会把任务提交到这个线程池中。新版本中这个连接池的使用频率不是很大了,可以设置一个较小的值。",
"指定使用Mycat全局序列的类型。",
"主要用于指定NIOProcessor上共享的businessExecutor固定线程池大小。Server在需要处理一些异步逻辑的时候会把任务提交到这个线程池中。新版本中这个连接池的使用频率不是很大了,可以设置一个较小的值。",
"指定使用全局序列的类型。",
"指定Mysql协议中的报文头长度。默认4",
"指定Mysql协议可以携带的数据最大长度。默认16M",
"指定连接的空闲超时时间。某连接在发起空闲检查下发现距离上次使用超过了空闲时间那么这个连接会被回收就是被直接的关闭掉。默认30分钟",
"连接的初始化字符集。默认为utf8",
"前端连接的初始化事务隔离级别只在初始化的时候使用后续会根据客户端传递过来的属性对后端数据库连接进行同步。默认为REPEATED_READ",
"SQL执行超时的时间Mycat会检查连接上最后一次执行SQL的时间若超过这个时间则会直接关闭这连接。默认时间为300秒",
"SQL执行超时的时间Server会检查连接上最后一次执行SQL的时间若超过这个时间则会直接关闭这连接。默认时间为300秒",
"清理NIOProcessor上前后端空闲、超时和关闭连接的间隔时间。默认是1秒",
"对后端连接进行空闲、超时检查的时间间隔默认是300秒",
"对后端所有读、写库发起心跳的间隔时间默认是10秒",
"mycat服务监听的IP地址默认值为0.0.0.0",
"mycat的使用端口默认值为8066",
"mycat的管理端口默认值为9066"};
"服务监听的IP地址默认值为0.0.0.0",
"使用端口默认值为8066",
"管理端口默认值为9066"};
public static final String[] ISOLATIONS = {"", "READ_UNCOMMITTED", "READ_COMMITTED", "REPEATED_READ", "SERIALIZABLE"};
}

View File

@@ -90,35 +90,35 @@ public class MyCatMemory {
*/
if(system.getUseOffHeapForMerge()== 1){
conf.set("mycat.memory.offHeap.enabled","true");
conf.set("server.memory.offHeap.enabled","true");
}else{
conf.set("mycat.memory.offHeap.enabled","false");
conf.set("server.memory.offHeap.enabled","false");
}
if(system.getUseStreamOutput() == 1){
conf.set("mycat.stream.output.result","true");
conf.set("server.stream.output.result","true");
}else{
conf.set("mycat.stream.output.result","false");
conf.set("server.stream.output.result","false");
}
if(system.getMemoryPageSize() != null){
conf.set("mycat.buffer.pageSize",system.getMemoryPageSize());
conf.set("server.buffer.pageSize",system.getMemoryPageSize());
}else{
conf.set("mycat.buffer.pageSize","1m");
conf.set("server.buffer.pageSize","1m");
}
if(system.getSpillsFileBufferSize() != null){
conf.set("mycat.merge.file.buffer",system.getSpillsFileBufferSize());
conf.set("server.merge.file.buffer",system.getSpillsFileBufferSize());
}else{
conf.set("mycat.merge.file.buffer","32k");
conf.set("server.merge.file.buffer","32k");
}
conf.set("mycat.pointer.array.len","8k")
.set("mycat.memory.offHeap.size", JavaUtils.bytesToString2(resultSetBufferSize));
conf.set("server.pointer.array.len","8k")
.set("server.memory.offHeap.size", JavaUtils.bytesToString2(resultSetBufferSize));
LOGGER.info("mycat.memory.offHeap.size: " +
LOGGER.info("resultSetBufferSize: " +
JavaUtils.bytesToString2(resultSetBufferSize));
resultMergeMemoryManager =
@@ -156,13 +156,13 @@ public class MyCatMemory {
* mycat.direct.output.result
* mycat.local.dir
*/
conf.set("mycat.memory.offHeap.enabled","true")
.set("mycat.pointer.array.len","8K")
.set("mycat.buffer.pageSize","1m")
.set("mycat.memory.offHeap.size", JavaUtils.bytesToString2(resultSetBufferSize))
.set("mycat.stream.output.result","false");
conf.set("server.memory.offHeap.enabled","true")
.set("server.pointer.array.len","8K")
.set("server.buffer.pageSize","1m")
.set("server.memory.offHeap.size", JavaUtils.bytesToString2(resultSetBufferSize))
.set("server.stream.output.result","false");
LOGGER.info("mycat.memory.offHeap.size: " + JavaUtils.bytesToString2(resultSetBufferSize));
LOGGER.info("resultSetBufferSize: " + JavaUtils.bytesToString2(resultSetBufferSize));
resultMergeMemoryManager =
new ResultMergeMemoryManager(conf,numCores,maxOnHeapMemory);

View File

@@ -1,338 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.mycat.memory.environment;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.InputStream;
import java.lang.management.ManagementFactory;
import java.lang.management.OperatingSystemMXBean;
import java.lang.management.RuntimeMXBean;
import java.lang.reflect.Method;
import java.util.List;
import java.util.Properties;
/**
* Utility class that gives access to the execution environment of the JVM, like
* the executing user, startup options, or the JVM version.
*/
public class EnvironmentInformation {
private static final Logger LOG = LoggerFactory.getLogger(EnvironmentInformation.class);
public static final String UNKNOWN = "<unknown>";
/**
* Returns the version of the code as String. If version == null, then the JobManager does not run from a
* Maven build. An example is a source code checkout, compile, and run from inside an IDE.
*
* @return The version string.
*/
public static String getVersion() {
String version = EnvironmentInformation.class.getPackage().getImplementationVersion();
return version != null ? version : UNKNOWN;
}
/**
* Returns the code revision (commit and commit date) of Flink, as generated by the Maven builds.
*
* @return The code revision.
*/
public static RevisionInformation getRevisionInformation() {
String revision = UNKNOWN;
String commitDate = UNKNOWN;
try (InputStream propFile = EnvironmentInformation.class.getClassLoader().getResourceAsStream(".version.properties")) {
if (propFile != null) {
Properties properties = new Properties();
properties.load(propFile);
String propRevision = properties.getProperty("git.commit.id.abbrev");
String propCommitDate = properties.getProperty("git.commit.time");
revision = propRevision != null ? propRevision : UNKNOWN;
commitDate = propCommitDate != null ? propCommitDate : UNKNOWN;
}
} catch (Throwable t) {
if (LOG.isDebugEnabled()) {
LOG.debug("Cannot determine code revision: Unable to read version property file.", t);
} else {
LOG.info("Cannot determine code revision: Unable to read version property file.");
}
}
return new RevisionInformation(revision, commitDate);
}
/**
* Gets the name of the user that is running the JVM.
*
* @return The name of the user that is running the JVM.
*/
public static String getUserRunning() {
String user = System.getProperty("user.name");
if (user == null) {
user = UNKNOWN;
if (LOG.isDebugEnabled()) {
LOG.debug("Cannot determine user/group information for the current user.");
}
}
return user;
}
/**
* The maximum JVM heap size, in bytes.
*
* @return The maximum JVM heap size, in bytes.
*/
public static long getMaxJvmHeapMemory() {
long maxMemory = Runtime.getRuntime().maxMemory();
if (maxMemory == Long.MAX_VALUE) {
// amount of free memory unknown
try {
// workaround for Oracle JDK
OperatingSystemMXBean operatingSystemMXBean = ManagementFactory.getOperatingSystemMXBean();
Class<?> clazz = Class.forName("com.sun.management.OperatingSystemMXBean");
Method method = clazz.getMethod("getTotalPhysicalMemorySize");
maxMemory = (Long) method.invoke(operatingSystemMXBean) / 4;
}
catch (Throwable e) {
throw new RuntimeException("Could not determine the amount of free memory.\n" +
"Please set the maximum memory for the JVM, e.g. -Xmx512M for 512 megabytes.");
}
}
return maxMemory;
}
/**
* Gets an estimate of the size of the free heap memory.
*
* NOTE: This method is heavy-weight. It triggers a garbage collection to reduce fragmentation and get
* a better estimate at the size of free memory. It is typically more accurate than the plain version
* {@link #getSizeOfFreeHeapMemory()}.
*
* @return An estimate of the size of the free heap memory, in bytes.
*/
public static long getSizeOfFreeHeapMemoryWithDefrag() {
// trigger a garbage collection, to reduce fragmentation
System.gc();
return getSizeOfFreeHeapMemory();
}
/**
* Gets an estimate of the size of the free heap memory. The estimate may vary, depending on the current
* level of memory fragmentation and the number of dead objects. For a better (but more heavy-weight)
* estimate, use {@link #getSizeOfFreeHeapMemoryWithDefrag()}.
*
* @return An estimate of the size of the free heap memory, in bytes.
*/
public static long getSizeOfFreeHeapMemory() {
Runtime r = Runtime.getRuntime();
long maxMemory = r.maxMemory();
if (maxMemory == Long.MAX_VALUE) {
// amount of free memory unknown
try {
// workaround for Oracle JDK
OperatingSystemMXBean operatingSystemMXBean = ManagementFactory.getOperatingSystemMXBean();
Class<?> clazz = Class.forName("com.sun.management.OperatingSystemMXBean");
Method method = clazz.getMethod("getTotalPhysicalMemorySize");
maxMemory = (Long) method.invoke(operatingSystemMXBean) / 4;
} catch (Throwable e) {
throw new RuntimeException("Could not determine the amount of free memory.\n" +
"Please set the maximum memory for the JVM, e.g. -Xmx512M for 512 megabytes.");
}
}
return maxMemory - r.totalMemory() + r.freeMemory();
}
/**
* Gets the version of the JVM in the form "VM_Name - Vendor - Spec/Version".
*
* @return The JVM version.
*/
public static String getJvmVersion() {
try {
final RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
return bean.getVmName() + " - " + bean.getVmVendor() + " - " + bean.getSpecVersion() + '/' + bean.getVmVersion();
}
catch (Throwable t) {
return UNKNOWN;
}
}
/**
* Gets the system parameters and environment parameters that were passed to the JVM on startup.
*
* @return The options passed to the JVM on startup.
*/
public static String getJvmStartupOptions() {
try {
final RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
final StringBuilder bld = new StringBuilder();
for (String s : bean.getInputArguments()) {
bld.append(s).append(' ');
}
return bld.toString();
}
catch (Throwable t) {
return UNKNOWN;
}
}
/**
* Gets the system parameters and environment parameters that were passed to the JVM on startup.
*
* @return The options passed to the JVM on startup.
*/
public static String[] getJvmStartupOptionsArray() {
try {
RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
List<String> options = bean.getInputArguments();
return options.toArray(new String[options.size()]);
}
catch (Throwable t) {
return new String[0];
}
}
/**
* Gets the directory for temporary files, as returned by the JVM system property "java.io.tmpdir".
*
* @return The directory for temporary files.
*/
public static String getTemporaryFileDirectory() {
return System.getProperty("java.io.tmpdir");
}
/**
* Tries to retrieve the maximum number of open file handles. This method will only work on
* UNIX-based operating systems with Sun/Oracle Java versions.
*
* <p>If the number of max open file handles cannot be determined, this method returns {@code -1}.</p>
*
* @return The limit of open file handles, or {@code -1}, if the limit could not be determined.
*/
public static long getOpenFileHandlesLimit() {
Class<?> sunBeanClass;
try {
sunBeanClass = Class.forName("com.sun.management.UnixOperatingSystemMXBean");
}
catch (ClassNotFoundException e) {
return -1L;
}
try {
Method fhLimitMethod = sunBeanClass.getMethod("getMaxFileDescriptorCount");
Object result = fhLimitMethod.invoke(ManagementFactory.getOperatingSystemMXBean());
return (Long) result;
}
catch (Throwable t) {
LOG.warn("Unexpected error when accessing file handle limit", t);
return -1L;
}
}
/**
* Logs a information about the environment, like code revision, current user, java version,
* and JVM parameters.
*
* @param log The logger to log the information to.
* @param componentName The component name to mention in the log.
* @param commandLineArgs The arguments accompanying the starting the component.
*/
public static void logEnvironmentInfo(Logger log, String componentName, String[] commandLineArgs) {
if (log.isInfoEnabled()) {
RevisionInformation rev = getRevisionInformation();
String version = getVersion();
String user = getUserRunning();
String jvmVersion = getJvmVersion();
String[] options = getJvmStartupOptionsArray();
String javaHome = System.getenv("JAVA_HOME");
long maxHeapMegabytes = getMaxJvmHeapMemory() >>> 20;
log.info("--------------------------------------------------------------------------------");
log.info(" Starting " + componentName + " (Version: " + version + ", "
+ "Rev:" + rev.commitId + ", " + "Date:" + rev.commitDate + ")");
log.info(" Current user: " + user);
log.info(" JVM: " + jvmVersion);
log.info(" Maximum heap size: " + maxHeapMegabytes + " MiBytes");
log.info(" JAVA_HOME: " + (javaHome == null ? "(not set)" : javaHome));
if (options.length == 0) {
log.info(" JVM Options: (none)");
}
else {
log.info(" JVM Options:");
for (String s: options) {
log.info(" " + s);
}
}
if (commandLineArgs == null || commandLineArgs.length == 0) {
log.info(" Program Arguments: (none)");
}
else {
log.info(" Program Arguments:");
for (String s: commandLineArgs) {
log.info(" " + s);
}
}
log.info(" Classpath: " + System.getProperty("java.class.path"));
log.info("--------------------------------------------------------------------------------");
}
}
// --------------------------------------------------------------------------------------------
/** Don't instantiate this class */
private EnvironmentInformation() {}
// --------------------------------------------------------------------------------------------
/**
* Revision information encapsulates information about the source code revision of the Flink
* code.
*/
public static class RevisionInformation {
/** The git commit id (hash) */
public final String commitId;
/** The git commit date */
public final String commitDate;
public RevisionInformation(String commitId, String commitDate) {
this.commitId = commitId;
this.commitDate = commitDate;
}
}
}

View File

@@ -58,26 +58,20 @@ public class Hardware {
* the size could not be determined
*/
public static long getSizeOfPhysicalMemory() {
switch (OperatingSystem.getCurrentOperatingSystem()) {
case LINUX:
return getSizeOfPhysicalMemoryForLinux();
case WINDOWS:
return getSizeOfPhysicalMemoryForWindows();
case MAC_OS:
return getSizeOfPhysicalMemoryForMac();
case FREE_BSD:
return getSizeOfPhysicalMemoryForFreeBSD();
case UNKNOWN:
LOG.error("Cannot determine size of physical memory for unknown operating system");
return -1;
default:
LOG.error("Unrecognized OS: " + OperatingSystem.getCurrentOperatingSystem());
return -1;
if (OperatingSystem.getCurrentOperatingSystem() == OperatingSystem.LINUX) {
return getSizeOfPhysicalMemoryForLinux();
} else if (OperatingSystem.getCurrentOperatingSystem() == OperatingSystem.WINDOWS) {
return getSizeOfPhysicalMemoryForWindows();
} else if (OperatingSystem.getCurrentOperatingSystem() == OperatingSystem.MAC_OS) {
return getSizeOfPhysicalMemoryForMac();
} else if (OperatingSystem.getCurrentOperatingSystem() == OperatingSystem.FREE_BSD) {
return getSizeOfPhysicalMemoryForFreeBSD();
} else if (OperatingSystem.getCurrentOperatingSystem() == OperatingSystem.UNKNOWN) {
LOG.error("Cannot determine size of physical memory for unknown operating system");
return -1;
} else {
LOG.error("Unrecognized OS: " + OperatingSystem.getCurrentOperatingSystem());
return -1;
}
}

View File

@@ -280,8 +280,8 @@ public final class UnsafeFixedWidthAggregationMap {
return new UnsafeKVExternalSorter(
groupingKeySchema,
aggregationBufferSchema,
MycatServer.getInstance().getMyCatMemory().getBlockManager(),
MycatServer.getInstance().getMyCatMemory().getSerializerManager(),
MycatServer.getInstance().getServerMemory().getBlockManager(),
MycatServer.getInstance().getServerMemory().getSerializerManager(),
map.getPageSizeBytes(),
map);
}

View File

@@ -29,7 +29,7 @@ public abstract class MemoryManager {
public MemoryManager(MycatPropertyConf conf, int numCores, long onHeapExecutionMemory){
this.conf = conf;
this.numCores =numCores;
maxOffHeapMemory = conf.getSizeAsBytes("mycat.memory.offHeap.size","128m");
maxOffHeapMemory = conf.getSizeAsBytes("server.memory.offHeap.size","128m");
offHeapExecutionMemory = maxOffHeapMemory;
onHeapExecutionMemoryPool.incrementPoolSize(onHeapExecutionMemory);
@@ -43,13 +43,12 @@ public abstract class MemoryManager {
*/
public void releaseExecutionMemory(long numBytes, long taskAttemptId, MemoryMode memoryMode) {
synchronized (this) {
switch (memoryMode) {
case ON_HEAP:
onHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId);
break;
case OFF_HEAP:
offHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId);
break;
if (memoryMode == MemoryMode.ON_HEAP) {
onHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId);
} else if (memoryMode == MemoryMode.OFF_HEAP) {
offHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId);
}
}
@@ -91,8 +90,8 @@ public void releaseExecutionMemory(long numBytes, long taskAttemptId, MemoryMode
* sun.misc.Unsafe.
*/
public final MemoryMode tungstenMemoryMode(){
if (conf.getBoolean("mycat.memory.offHeap.enabled", false)) {
assert (conf.getSizeAsBytes("mycat.memory.offHeap.size",0) > 0);
if (conf.getBoolean("server.memory.offHeap.enabled", false)) {
assert (conf.getSizeAsBytes("server.memory.offHeap.size",0) > 0);
assert (Platform.unaligned());
return MemoryMode.OFF_HEAP;
} else {
@@ -103,7 +102,7 @@ public void releaseExecutionMemory(long numBytes, long taskAttemptId, MemoryMode
/**
* The default page size, in bytes.
*
* If user didn't explicitly set "mycat.buffer.pageSize", we figure out the default value
* If user didn't explicitly set "server.buffer.pageSize", we figure out the default value
* by looking at the number of cores available to the process, and the total amount of memory,
* and then divide it by a factor of safety.
*/
@@ -124,22 +123,22 @@ public void releaseExecutionMemory(long numBytes, long taskAttemptId, MemoryMode
int safetyFactor = 16;
long maxTungstenMemory = 0L;
switch (tungstenMemoryMode()){
case ON_HEAP:
synchronized (this) {
maxTungstenMemory = onHeapExecutionMemoryPool.poolSize();
}
break;
case OFF_HEAP:
synchronized (this) {
maxTungstenMemory = offHeapExecutionMemoryPool.poolSize();
}
break;
MemoryMode i = tungstenMemoryMode();
if (i == MemoryMode.ON_HEAP) {
synchronized (this) {
maxTungstenMemory = onHeapExecutionMemoryPool.poolSize();
}
} else if (i == MemoryMode.OFF_HEAP) {
synchronized (this) {
maxTungstenMemory = offHeapExecutionMemoryPool.poolSize();
}
}
long size = ByteArrayMethods.nextPowerOf2(maxTungstenMemory / cores / safetyFactor);
long defaultSize = Math.min(maxPageSize, Math.max(minPageSize, size));
defaultSize = conf.getSizeAsBytes("mycat.buffer.pageSize", defaultSize);
defaultSize = conf.getSizeAsBytes("server.buffer.pageSize", defaultSize);
return defaultSize;
}
@@ -148,11 +147,11 @@ public void releaseExecutionMemory(long numBytes, long taskAttemptId, MemoryMode
* Allocates memory for use by Unsafe/Tungsten code.
*/
public final MemoryAllocator tungstenMemoryAllocator() {
switch (tungstenMemoryMode()){
case ON_HEAP:
return MemoryAllocator.HEAP;
case OFF_HEAP:
return MemoryAllocator.UNSAFE;
MemoryMode i = tungstenMemoryMode();
if (i == MemoryMode.ON_HEAP) {
return MemoryAllocator.HEAP;
} else if (i == MemoryMode.OFF_HEAP) {
return MemoryAllocator.UNSAFE;
}
return null;
}

View File

@@ -20,11 +20,10 @@ public class ResultMergeMemoryManager extends MemoryManager {
@Override
protected synchronized long acquireExecutionMemory(long numBytes,long taskAttemptId,MemoryMode memoryMode) throws InterruptedException {
switch (memoryMode) {
case ON_HEAP:
return onHeapExecutionMemoryPool.acquireMemory(numBytes,taskAttemptId);
case OFF_HEAP:
return offHeapExecutionMemoryPool.acquireMemory(numBytes,taskAttemptId);
if (memoryMode == MemoryMode.ON_HEAP) {
return onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId);
} else if (memoryMode == MemoryMode.OFF_HEAP) {
return offHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId);
}
return 0L;
}

View File

@@ -27,11 +27,10 @@ public class ResultSetMemoryPool extends MemoryPool {
private String poolName(){
switch (memoryMode){
case ON_HEAP:
return "on-heap memory";
case OFF_HEAP:
return "off-heap memory";
if (memoryMode == MemoryMode.ON_HEAP) {
return "on-heap memory";
} else if (memoryMode == MemoryMode.OFF_HEAP) {
return "off-heap memory";
}
return "off-heap memory";

View File

@@ -38,7 +38,7 @@ public class DataNodeDiskManager {
File file,
SerializerInstance serializerInstance,
int bufferSize) throws IOException {
boolean syncWrites = conf.getBoolean("mycat.merge.sync", false);
boolean syncWrites = conf.getBoolean("server.merge.sync", false);
return new DiskRowWriter(file, serializerInstance, bufferSize,new FileOutputStream(file),
syncWrites,blockId);
}

View File

@@ -65,7 +65,7 @@ public class DataNodeFileManager {
this.deleteFilesOnStop = deleteFilesOnStop;
subDirsPerLocalDir = conf.getInt("mycat.diskStore.subDirectories", 64);
subDirsPerLocalDir = conf.getInt("server.diskStore.subDirectories", 64);
localDirs = createLocalDirs(conf);
if (localDirs.isEmpty()) {
System.exit(-1);
@@ -138,7 +138,7 @@ public class DataNodeFileManager {
*/
private List<File> createLocalDirs(MycatPropertyConf conf) {
String rootDirs = conf.getString("mycat.local.dirs","datanode");
String rootDirs = conf.getString("server.local.dirs","datanode");
String rdir[] = rootDirs.split(",");
List<File> dirs = new ArrayList<File>();

View File

@@ -50,7 +50,7 @@ public final class UnsafeExternalRowSorter {
}
public UnsafeExternalRowSorter(DataNodeMemoryManager dataNodeMemoryManager,
@Nonnull MyCatMemory myCatMemory,
@Nonnull MyCatMemory memory,
StructType schema,
PrefixComparator prefixComparator,
PrefixComputer prefixComputer,
@@ -63,11 +63,11 @@ public final class UnsafeExternalRowSorter {
this.recordComparator = new RowComparator(schema);
sorter = UnsafeExternalSorter.create(
dataNodeMemoryManager,
myCatMemory.getBlockManager(),
myCatMemory.getSerializerManager(),
memory.getBlockManager(),
memory.getSerializerManager(),
recordComparator,
prefixComparator,
myCatMemory.getConf().getSizeAsBytes("mycat.pointer.array.len","16K"),
memory.getConf().getSizeAsBytes("server.pointer.array.len","16K"),
pageSizeBytes,
canUseRadixSort,
enableSort);

View File

@@ -120,9 +120,9 @@ public final class UnsafeExternalSorter extends MemoryConsumer {
this.prefixComparator = prefixComparator;
if(MycatServer.getInstance().getMyCatMemory() != null){
if(MycatServer.getInstance().getServerMemory() != null){
this.fileBufferSizeBytes = (int) MycatServer.getInstance().
getMyCatMemory().getConf().getSizeAsBytes("mycat.merge.file.buffer", "32k");
getServerMemory().getConf().getSizeAsBytes("server.merge.file.buffer", "32k");
}else{
this.fileBufferSizeBytes = 32*1024;
}

View File

@@ -24,9 +24,9 @@ import io.mycat.config.model.DBHostConfig;
import io.mycat.config.model.SchemaConfig;
import io.mycat.config.model.SystemConfig;
import io.mycat.config.model.TableConfig;
import io.mycat.meta.protocol.MyCatMeta.ColumnMeta;
import io.mycat.meta.protocol.MyCatMeta.IndexMeta;
import io.mycat.meta.protocol.MyCatMeta.TableMeta;
import io.mycat.meta.protocol.StructureMeta.ColumnMeta;
import io.mycat.meta.protocol.StructureMeta.IndexMeta;
import io.mycat.meta.protocol.StructureMeta.TableMeta;
import io.mycat.meta.table.AbstractTableMetaHandler;
import io.mycat.meta.table.MetaHelper;
import io.mycat.meta.table.MetaHelper.INDEX_TYPE;
@@ -523,7 +523,7 @@ public class ProxyMetaManager {
} catch (Exception e) {
LOGGER.warn("updateMetaData failed,sql is" + statement.toString(), e);
} finally {
removeMetaLock(schema, schemaInfo.table);
removeMetaLock(schemaInfo.schema, schemaInfo.table);
if (MycatServer.getInstance().isUseZK()) {
try {
notifyClusterDDL(schemaInfo.schema, schemaInfo.table, sql, isSuccess ? DDLStatus.SUCCESS : DDLStatus.FAILED);
@@ -548,7 +548,7 @@ public class ProxyMetaManager {
} catch (Exception e) {
LOGGER.warn("updateMetaData failed,sql is" + statement.toString(), e);
} finally {
removeMetaLock(schema, schemaInfo.table);
removeMetaLock(schemaInfo.schema, schemaInfo.table);
if (MycatServer.getInstance().isUseZK()) {
try {
@@ -665,7 +665,7 @@ public class ProxyMetaManager {
} catch (Exception e) {
LOGGER.warn("updateMetaData alterTable failed,sql is" + alterStatement.toString(), e);
} finally {
removeMetaLock(schema, schemaInfo.table);
removeMetaLock(schemaInfo.schema, schemaInfo.table);
if (MycatServer.getInstance().isUseZK()) {
try {
notifyClusterDDL(schemaInfo.schema, schemaInfo.table, sql, isSuccess ? DDLStatus.SUCCESS : DDLStatus.FAILED);
@@ -697,7 +697,7 @@ public class ProxyMetaManager {
} catch (Exception e) {
LOGGER.warn("updateMetaData failed,sql is" + statement.toString(), e);
} finally {
removeMetaLock(schema, schemaInfo.table);
removeMetaLock(schemaInfo.schema, schemaInfo.table);
if (MycatServer.getInstance().isUseZK()) {
try {
notifyClusterDDL(schemaInfo.schema, schemaInfo.table, sql, isSuccess ? DDLStatus.SUCCESS : DDLStatus.FAILED);
@@ -750,7 +750,7 @@ public class ProxyMetaManager {
} catch (Exception e) {
LOGGER.warn("updateMetaData failed,sql is" + dropIndexStatement.toString(), e);
} finally {
removeMetaLock(schema, schemaInfo.table);
removeMetaLock(schemaInfo.schema, schemaInfo.table);
if (MycatServer.getInstance().isUseZK()) {
try {
notifyClusterDDL(schemaInfo.schema, schemaInfo.table, sql, isSuccess ? DDLStatus.SUCCESS : DDLStatus.FAILED);

View File

@@ -6,7 +6,7 @@ import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import io.mycat.meta.protocol.MyCatMeta.TableMeta;
import io.mycat.meta.protocol.StructureMeta.TableMeta;
public class SchemaMeta {

View File

@@ -1,7 +1,8 @@
syntax = "proto2";
package io.mycat.meta.protocol;
option java_package = "io.mycat.meta.protocol";
option java_outer_classname = "MyCatMeta";
option java_outer_classname = "StructureMeta";
option optimize_for = SPEED;

View File

@@ -19,7 +19,7 @@ import com.alibaba.druid.sql.parser.SQLStatementParser;
import io.mycat.MycatServer;
import io.mycat.backend.datasource.PhysicalDBNode;
import io.mycat.config.model.TableConfig;
import io.mycat.meta.protocol.MyCatMeta.TableMeta;
import io.mycat.meta.protocol.StructureMeta.TableMeta;
import io.mycat.sqlengine.OneRawSQLQueryResultHandler;
import io.mycat.sqlengine.SQLJob;
import io.mycat.sqlengine.SQLQueryResult;

View File

@@ -21,9 +21,9 @@ import com.alibaba.druid.sql.dialect.mysql.ast.expr.MySqlOrderingExpr;
import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlTableIndex;
import com.alibaba.druid.sql.dialect.mysql.visitor.MySqlOutputVisitor;
import io.mycat.meta.protocol.MyCatMeta.ColumnMeta;
import io.mycat.meta.protocol.MyCatMeta.IndexMeta;
import io.mycat.meta.protocol.MyCatMeta.TableMeta;
import io.mycat.meta.protocol.StructureMeta.ColumnMeta;
import io.mycat.meta.protocol.StructureMeta.IndexMeta;
import io.mycat.meta.protocol.StructureMeta.TableMeta;
import io.mycat.util.StringUtil;
public class MetaHelper {

View File

@@ -3,7 +3,7 @@ package io.mycat.meta.table;
import io.mycat.MycatServer;
import io.mycat.config.model.TableConfig;
import io.mycat.meta.ProxyMetaManager;
import io.mycat.meta.protocol.MyCatMeta.TableMeta;
import io.mycat.meta.protocol.StructureMeta.TableMeta;
import java.util.Set;

View File

@@ -2,7 +2,7 @@ package io.mycat.meta.table;
import io.mycat.MycatServer;
import io.mycat.config.model.TableConfig;
import io.mycat.meta.protocol.MyCatMeta.TableMeta;
import io.mycat.meta.protocol.StructureMeta.TableMeta;
import java.util.Set;

View File

@@ -102,7 +102,7 @@ public final class AIOAcceptor implements SocketAcceptor,
serverChannel.accept(ID_GENERATOR.getId(), this);
} else {
throw new IllegalStateException(
"MyCAT Server Channel has been closed");
"Server Channel has been closed");
}
}

View File

@@ -70,7 +70,7 @@ public class HeartbeatPacket extends MySQLPacket {
@Override
protected String getPacketInfo() {
return "Mycat Heartbeat Packet";
return "Heartbeat Packet";
}
}

View File

@@ -51,59 +51,52 @@ public abstract class Field {
public static Field getFieldItem(String name, String table, int type, int charsetIndex, int field_length,
int decimals, long flags) {
FieldTypes field_type = FieldTypes.valueOf(type);
switch (field_type) {
case MYSQL_TYPE_NEWDECIMAL:
// mysql use newdecimal after some version
if (field_type == FieldTypes.MYSQL_TYPE_NEWDECIMAL) {// mysql use newdecimal after some version
return new FieldNewdecimal(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_DECIMAL:
} else if (field_type == FieldTypes.MYSQL_TYPE_DECIMAL) {
return new FieldDecimal(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_TINY:
} else if (field_type == FieldTypes.MYSQL_TYPE_TINY) {
return new FieldTiny(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_SHORT:
} else if (field_type == FieldTypes.MYSQL_TYPE_SHORT) {
return new FieldShort(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_LONG:
} else if (field_type == FieldTypes.MYSQL_TYPE_LONG) {
return new FieldLong(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_FLOAT:
} else if (field_type == FieldTypes.MYSQL_TYPE_FLOAT) {
return new FieldFloat(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_DOUBLE:
} else if (field_type == FieldTypes.MYSQL_TYPE_DOUBLE) {
return new FieldDouble(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_NULL:
} else if (field_type == FieldTypes.MYSQL_TYPE_NULL) {
return FieldNull.getInstance();
case MYSQL_TYPE_TIMESTAMP:
} else if (field_type == FieldTypes.MYSQL_TYPE_TIMESTAMP) {
return new FieldTimestamp(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_LONGLONG:
} else if (field_type == FieldTypes.MYSQL_TYPE_LONGLONG) {
return new FieldLonglong(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_INT24:
} else if (field_type == FieldTypes.MYSQL_TYPE_INT24) {
return new FieldMedium(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_DATE:
case MYSQL_TYPE_NEWDATE:
} else if (field_type == FieldTypes.MYSQL_TYPE_DATE || field_type == FieldTypes.MYSQL_TYPE_NEWDATE) {
return new FieldDate(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_TIME:
} else if (field_type == FieldTypes.MYSQL_TYPE_TIME) {
return new FieldTime(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_DATETIME:
} else if (field_type == FieldTypes.MYSQL_TYPE_DATETIME) {
return new FieldDatetime(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_YEAR:
} else if (field_type == FieldTypes.MYSQL_TYPE_YEAR) {
return new FieldYear(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_VARCHAR:
} else if (field_type == FieldTypes.MYSQL_TYPE_VARCHAR) {
return new FieldVarchar(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_BIT:
} else if (field_type == FieldTypes.MYSQL_TYPE_BIT) {
return new FieldBit(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_VAR_STRING:
} else if (field_type == FieldTypes.MYSQL_TYPE_VAR_STRING) {
return new FieldVarstring(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_STRING:
} else if (field_type == FieldTypes.MYSQL_TYPE_STRING) {
return new FieldString(name, table, charsetIndex, field_length, decimals, flags);
/** --下列的类型函数目前不支持因为select *出来的mysql都转化成string了无法知晓它们在数据库中的type-- **/
case MYSQL_TYPE_ENUM:
/** --下列的类型函数目前不支持因为select *出来的mysql都转化成string了无法知晓它们在数据库中的type-- **/
} else if (field_type == FieldTypes.MYSQL_TYPE_ENUM) {
return new FieldEnum(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_SET:
} else if (field_type == FieldTypes.MYSQL_TYPE_SET) {
return new FieldSet(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_TINY_BLOB:
case MYSQL_TYPE_MEDIUM_BLOB:
case MYSQL_TYPE_LONG_BLOB:
case MYSQL_TYPE_BLOB:
} else if (field_type == FieldTypes.MYSQL_TYPE_TINY_BLOB || field_type == FieldTypes.MYSQL_TYPE_MEDIUM_BLOB || field_type == FieldTypes.MYSQL_TYPE_LONG_BLOB || field_type == FieldTypes.MYSQL_TYPE_BLOB) {
return new FieldBlob(name, table, charsetIndex, field_length, decimals, flags);
case MYSQL_TYPE_GEOMETRY:
default:
} else {
throw new RuntimeException("unsupported field type :" + field_type.toString() + "!");
}
}

View File

@@ -93,14 +93,9 @@ public class FieldUtil {
}
public static boolean is_temporal_type(FieldTypes valuetype) {
switch (valuetype) {
case MYSQL_TYPE_DATE:
case MYSQL_TYPE_DATETIME:
case MYSQL_TYPE_TIMESTAMP:
case MYSQL_TYPE_TIME:
case MYSQL_TYPE_NEWDATE:
if (valuetype == FieldTypes.MYSQL_TYPE_DATE || valuetype == FieldTypes.MYSQL_TYPE_DATETIME || valuetype == FieldTypes.MYSQL_TYPE_TIMESTAMP || valuetype == FieldTypes.MYSQL_TYPE_TIME || valuetype == FieldTypes.MYSQL_TYPE_NEWDATE) {
return true;
default:
} else {
return false;
}
}
@@ -115,34 +110,25 @@ public class FieldUtil {
* @retval false If field real type is not temporal
*/
public static boolean is_temporal_real_type(FieldTypes type) {
switch (type) {
case MYSQL_TYPE_TIME2:
case MYSQL_TYPE_TIMESTAMP2:
case MYSQL_TYPE_DATETIME2:
if (type == FieldTypes.MYSQL_TYPE_TIME2 || type == FieldTypes.MYSQL_TYPE_TIMESTAMP2 || type == FieldTypes.MYSQL_TYPE_DATETIME2) {
return true;
default:
} else {
return FieldUtil.is_temporal_type(type);
}
}
public static boolean is_temporal_type_with_time(FieldTypes type) {
switch (type) {
case MYSQL_TYPE_TIME:
case MYSQL_TYPE_DATETIME:
case MYSQL_TYPE_TIMESTAMP:
if (type == FieldTypes.MYSQL_TYPE_TIME || type == FieldTypes.MYSQL_TYPE_DATETIME || type == FieldTypes.MYSQL_TYPE_TIMESTAMP) {
return true;
default:
} else {
return false;
}
}
public static boolean is_temporal_type_with_date(FieldTypes valuetype) {
switch (valuetype) {
case MYSQL_TYPE_DATE:
case MYSQL_TYPE_DATETIME:
case MYSQL_TYPE_TIMESTAMP:
if (valuetype == FieldTypes.MYSQL_TYPE_DATE || valuetype == FieldTypes.MYSQL_TYPE_DATETIME || valuetype == FieldTypes.MYSQL_TYPE_TIMESTAMP) {
return true;
default:
} else {
return false;
}
}
@@ -158,11 +144,9 @@ public class FieldUtil {
* parts.
*/
public static boolean is_temporal_type_with_date_and_time(FieldTypes type) {
switch (type) {
case MYSQL_TYPE_DATETIME:
case MYSQL_TYPE_TIMESTAMP:
if (type == FieldTypes.MYSQL_TYPE_DATETIME || type == FieldTypes.MYSQL_TYPE_TIMESTAMP) {
return true;
default:
} else {
return false;
}
}
@@ -184,17 +168,16 @@ public class FieldUtil {
* @retval Field type.
*/
public static FieldTypes real_type_to_type(FieldTypes real_type) {
switch (real_type) {
case MYSQL_TYPE_TIME2:
if (real_type == FieldTypes.MYSQL_TYPE_TIME2) {
return FieldTypes.MYSQL_TYPE_TIME;
case MYSQL_TYPE_DATETIME2:
} else if (real_type == FieldTypes.MYSQL_TYPE_DATETIME2) {
return FieldTypes.MYSQL_TYPE_DATETIME;
case MYSQL_TYPE_TIMESTAMP2:
} else if (real_type == FieldTypes.MYSQL_TYPE_TIMESTAMP2) {
return FieldTypes.MYSQL_TYPE_TIMESTAMP;
case MYSQL_TYPE_NEWDATE:
} else if (real_type == FieldTypes.MYSQL_TYPE_NEWDATE) {
return FieldTypes.MYSQL_TYPE_DATE;
/* Note: NEWDECIMAL is a type, not only a real_type */
default:
} else {
return real_type;
}
}

View File

@@ -114,17 +114,16 @@ public abstract class Item {
}
public FieldTypes fieldType() {
switch (resultType()) {
case STRING_RESULT:
ItemResult i = resultType();
if (i == ItemResult.STRING_RESULT) {
return FieldTypes.MYSQL_TYPE_STRING;
case INT_RESULT:
} else if (i == ItemResult.INT_RESULT) {
return FieldTypes.MYSQL_TYPE_LONG;
case DECIMAL_RESULT:
} else if (i == ItemResult.DECIMAL_RESULT) {
return FieldTypes.MYSQL_TYPE_DECIMAL;
case REAL_RESULT:
} else if (i == ItemResult.REAL_RESULT) {
return FieldTypes.MYSQL_TYPE_DOUBLE;
case ROW_RESULT:
default:
} else {
return FieldTypes.MYSQL_TYPE_STRING;
}
}
@@ -194,22 +193,31 @@ public abstract class Item {
byte[] result = null;
FieldTypes f_type;
switch (f_type = fieldType()) {
default:
case MYSQL_TYPE_NULL:
case MYSQL_TYPE_DECIMAL:
case MYSQL_TYPE_ENUM:
case MYSQL_TYPE_SET:
case MYSQL_TYPE_TINY_BLOB:
case MYSQL_TYPE_MEDIUM_BLOB:
case MYSQL_TYPE_LONG_BLOB:
case MYSQL_TYPE_BLOB:
case MYSQL_TYPE_GEOMETRY:
case MYSQL_TYPE_STRING:
case MYSQL_TYPE_VAR_STRING:
case MYSQL_TYPE_VARCHAR:
case MYSQL_TYPE_BIT:
case MYSQL_TYPE_NEWDECIMAL: {
FieldTypes i = f_type = fieldType();
if (i == FieldTypes.MYSQL_TYPE_TINY || i == FieldTypes.MYSQL_TYPE_SHORT || i == FieldTypes.MYSQL_TYPE_YEAR || i == FieldTypes.MYSQL_TYPE_INT24 || i == FieldTypes.MYSQL_TYPE_LONG || i == FieldTypes.MYSQL_TYPE_LONGLONG) {
BigInteger bi = valInt();
if (!nullValue)
result = bi.toString().getBytes();
} else if (i == FieldTypes.MYSQL_TYPE_FLOAT || i == FieldTypes.MYSQL_TYPE_DOUBLE) {
BigDecimal bd = valReal();
if (!nullValue)
result = bd.toString().getBytes();
} else if (i == FieldTypes.MYSQL_TYPE_DATETIME || i == FieldTypes.MYSQL_TYPE_DATE || i == FieldTypes.MYSQL_TYPE_TIMESTAMP) {
MySQLTime tm = new MySQLTime();
getDate(tm, MyTime.TIME_FUZZY_DATE);
if (!nullValue) {
if (f_type == FieldTypes.MYSQL_TYPE_DATE) {
result = MyTime.my_date_to_str(tm).getBytes();
} else {
result = MyTime.my_datetime_to_str(tm, decimals).getBytes();
}
}
} else if (i == FieldTypes.MYSQL_TYPE_TIME) {
MySQLTime tm = new MySQLTime();
getTime(tm);
if (!nullValue)
result = MyTime.my_time_to_str(tm, decimals).getBytes();
} else {
String res = null;
if ((res = valStr()) != null)
try {
@@ -220,47 +228,11 @@ public abstract class Item {
else {
assert (nullValue);
}
break;
}
case MYSQL_TYPE_TINY:
case MYSQL_TYPE_SHORT:
case MYSQL_TYPE_YEAR:
case MYSQL_TYPE_INT24:
case MYSQL_TYPE_LONG:
case MYSQL_TYPE_LONGLONG: {
BigInteger bi = valInt();
if (!nullValue)
result = bi.toString().getBytes();
break;
}
case MYSQL_TYPE_FLOAT:
case MYSQL_TYPE_DOUBLE: {
BigDecimal bd = valReal();
if (!nullValue)
result = bd.toString().getBytes();
break;
}
case MYSQL_TYPE_DATETIME:
case MYSQL_TYPE_DATE:
case MYSQL_TYPE_TIMESTAMP: {
MySQLTime tm = new MySQLTime();
getDate(tm, MyTime.TIME_FUZZY_DATE);
if (!nullValue) {
if (f_type == FieldTypes.MYSQL_TYPE_DATE) {
result = MyTime.my_date_to_str(tm).getBytes();
} else {
result = MyTime.my_datetime_to_str(tm, decimals).getBytes();
}
}
break;
}
case MYSQL_TYPE_TIME: {
MySQLTime tm = new MySQLTime();
getTime(tm);
if (!nullValue)
result = MyTime.my_time_to_str(tm, decimals).getBytes();
break;
}
}
if (nullValue)
result = null;
@@ -268,20 +240,17 @@ public abstract class Item {
}
public boolean valBool() {
switch (resultType()) {
case INT_RESULT:
ItemResult i = resultType();
if (i == ItemResult.INT_RESULT) {
return valInt().longValue() != 0;
case DECIMAL_RESULT: {
} else if (i == ItemResult.DECIMAL_RESULT) {
BigDecimal val = valDecimal();
if (val != null)
return val.compareTo(BigDecimal.ZERO) != 0;
return false;
}
case REAL_RESULT:
case STRING_RESULT:
} else if (i == ItemResult.REAL_RESULT || i == ItemResult.STRING_RESULT) {
return !(valReal().compareTo(BigDecimal.ZERO) == 0);
case ROW_RESULT:
default:
} else {
return false; // Wrong (but safe)
}
}
@@ -316,17 +285,15 @@ public abstract class Item {
}
public Comparable getValueWithType(ItemResult type) {
switch (type) {
case REAL_RESULT:
if (type == ItemResult.REAL_RESULT) {
return valReal();
case DECIMAL_RESULT:
} else if (type == ItemResult.DECIMAL_RESULT) {
return valDecimal();
case INT_RESULT:
} else if (type == ItemResult.INT_RESULT) {
return valInt();
case STRING_RESULT:
} else if (type == ItemResult.STRING_RESULT) {
return valStr();
default:
break;
} else {
}
return null;
}
@@ -593,15 +560,14 @@ public abstract class Item {
* Convert a numeric type to date
*/
protected boolean getDateFromNumeric(MySQLTime ltime, long flags) {
switch (resultType()) {
case REAL_RESULT:
ItemResult i = resultType();
if (i == ItemResult.REAL_RESULT) {
return getDateFromReal(ltime, flags);
case DECIMAL_RESULT:
} else if (i == ItemResult.DECIMAL_RESULT) {
return getDateFromDecimal(ltime, flags);
case INT_RESULT:
} else if (i == ItemResult.INT_RESULT) {
return getDateFromInt(ltime, flags);
case STRING_RESULT:
case ROW_RESULT:
} else if (i == ItemResult.STRING_RESULT || i == ItemResult.ROW_RESULT) {
assert (false);
}
return (nullValue = true); // Impossible result_type
@@ -612,16 +578,16 @@ public abstract class Item {
*/
protected boolean getDateFromNonTemporal(MySQLTime ltime, long fuzzydate) {
assert (!isTemporal());
switch (resultType()) {
case STRING_RESULT:
ItemResult i = resultType();
if (i == ItemResult.STRING_RESULT) {
return getDateFromString(ltime, fuzzydate);
case REAL_RESULT:
} else if (i == ItemResult.REAL_RESULT) {
return getDateFromReal(ltime, fuzzydate);
case DECIMAL_RESULT:
} else if (i == ItemResult.DECIMAL_RESULT) {
return getDateFromDecimal(ltime, fuzzydate);
case INT_RESULT:
} else if (i == ItemResult.INT_RESULT) {
return getDateFromInt(ltime, fuzzydate);
case ROW_RESULT:
} else if (i == ItemResult.ROW_RESULT) {
assert (false);
}
return (nullValue = true); // Impossible result_type
@@ -701,15 +667,14 @@ public abstract class Item {
*/
protected boolean getTimeFromNumeric(MySQLTime ltime) {
assert (!isTemporal());
switch (resultType()) {
case REAL_RESULT:
ItemResult i = resultType();
if (i == ItemResult.REAL_RESULT) {
return getTimeFromReal(ltime);
case DECIMAL_RESULT:
} else if (i == ItemResult.DECIMAL_RESULT) {
return getTimeFromDecimal(ltime);
case INT_RESULT:
} else if (i == ItemResult.INT_RESULT) {
return getTimeFromInt(ltime);
case STRING_RESULT:
case ROW_RESULT:
} else if (i == ItemResult.STRING_RESULT || i == ItemResult.ROW_RESULT) {
assert (false);
}
return (nullValue = true); // Impossible result type
@@ -720,16 +685,16 @@ public abstract class Item {
*/
protected boolean getTimeFromNonTemporal(MySQLTime ltime) {
assert (!isTemporal());
switch (resultType()) {
case STRING_RESULT:
ItemResult i = resultType();
if (i == ItemResult.STRING_RESULT) {
return getTimeFromString(ltime);
case REAL_RESULT:
} else if (i == ItemResult.REAL_RESULT) {
return getTimeFromReal(ltime);
case DECIMAL_RESULT:
} else if (i == ItemResult.DECIMAL_RESULT) {
return getTimeFromDecimal(ltime);
case INT_RESULT:
} else if (i == ItemResult.INT_RESULT) {
return getTimeFromInt(ltime);
case ROW_RESULT:
} else if (i == ItemResult.ROW_RESULT) {
assert (false);
}
return (nullValue = true); // Impossible result type

View File

@@ -313,29 +313,25 @@ public class ItemCreate {
public ItemFunc create_func_cast(Item a, CastType type) {
CastTarget cast_type = type.target;
ItemFunc res = null;
switch (cast_type) {
case ITEM_CAST_BINARY:
res = new ItemFuncBinary(a,type.length);
break;
case ITEM_CAST_SIGNED_INT:
if (cast_type == CastTarget.ITEM_CAST_BINARY) {
res = new ItemFuncBinary(a, type.length);
} else if (cast_type == CastTarget.ITEM_CAST_SIGNED_INT) {
res = new ItemFuncSigned(a);
break;
case ITEM_CAST_UNSIGNED_INT:
} else if (cast_type == CastTarget.ITEM_CAST_UNSIGNED_INT) {
res = new ItemFuncUnsigned(a);
break;
case ITEM_CAST_DATE:
} else if (cast_type == CastTarget.ITEM_CAST_DATE) {
res = new ItemDateTypecast(a);
break;
case ITEM_CAST_TIME:
case ITEM_CAST_DATETIME: {
} else if (cast_type == CastTarget.ITEM_CAST_TIME || cast_type == CastTarget.ITEM_CAST_DATETIME) {
if (type.length > MyTime.DATETIME_MAX_DECIMALS)
throw new MySQLOutPutException(ErrorCode.ER_OPTIMIZER, "",
"too big precision in cast time/datetime,max 6,current:" + type.length);
res = (cast_type == CastTarget.ITEM_CAST_TIME) ? new ItemTimeTypecast(a, type.length)
: new ItemDatetimeTypecast(a, type.length);
break;
}
case ITEM_CAST_DECIMAL: {
} else if (cast_type == CastTarget.ITEM_CAST_DECIMAL) {
if (type.length < type.dec) {
throw new MySQLOutPutException(ErrorCode.ER_OPTIMIZER, "",
"For float(m,d), double(m,d) or decimal(m,d), M must be >= d");
@@ -349,19 +345,13 @@ public class ItemCreate {
"Too big scale " + type.dec + " max is " + MySQLcom.DECIMAL_MAX_SCALE);
}
res = new ItemDecimalTypecast(a, type.length, type.dec);
break;
}
case ITEM_CAST_NCHAR: {
} else if (cast_type == CastTarget.ITEM_CAST_NCHAR) {
int len = -1;
if (type.length > 0)
len = type.length;
res = new ItemNCharTypecast(a, len);
break;
}
default: {
} else {
assert (false);
break;
}
}
return res;
}

View File

@@ -23,20 +23,18 @@ public class ItemFuncCeiling extends ItemFuncIntVal {
@Override
public BigInteger intOp() {
BigInteger result;
switch (args.get(0).resultType()) {
case INT_RESULT:
ItemResult i = args.get(0).resultType();
if (i == ItemResult.INT_RESULT) {
result = args.get(0).valInt();
nullValue = args.get(0).nullValue;
break;
case DECIMAL_RESULT: {
} else if (i == ItemResult.DECIMAL_RESULT) {
BigDecimal dec = decimalOp();
if (dec == null)
result = BigInteger.ZERO;
else
result = dec.toBigInteger();
break;
}
default:
} else {
result = realOp().toBigInteger();
}
;

View File

@@ -23,20 +23,18 @@ public class ItemFuncFloor extends ItemFuncIntVal {
@Override
public BigInteger intOp() {
BigInteger result;
switch (args.get(0).resultType()) {
case INT_RESULT:
ItemResult i = args.get(0).resultType();
if (i == ItemResult.INT_RESULT) {
result = args.get(0).valInt();
nullValue = args.get(0).nullValue;
break;
case DECIMAL_RESULT: {
} else if (i == ItemResult.DECIMAL_RESULT) {
BigDecimal dec = decimalOp();
if (dec == null)
result = BigInteger.ZERO;
else
result = dec.toBigInteger();
break;
}
default:
} else {
result = realOp().toBigInteger();
}
;

View File

@@ -19,19 +19,18 @@ public abstract class ItemFuncIntVal extends ItemFuncNum1 {
@Override
public void findNumType() {
switch (hybrid_type = args.get(0).resultType()) {
case STRING_RESULT:
case REAL_RESULT:
ItemResult i = hybrid_type = args.get(0).resultType();
if (i == ItemResult.STRING_RESULT || i == ItemResult.REAL_RESULT) {
hybrid_type = ItemResult.REAL_RESULT;
maxLength = floatLength(decimals);
break;
case INT_RESULT:
} else if (i == ItemResult.INT_RESULT) {
hybrid_type = ItemResult.INT_RESULT;
break;
case DECIMAL_RESULT:
} else if (i == ItemResult.DECIMAL_RESULT) {
hybrid_type = ItemResult.DECIMAL_RESULT;
break;
default:
} else {
assert (false);
}
}

View File

@@ -78,26 +78,22 @@ public abstract class ItemFuncRoundOrTruncate extends ItemFuncNum1 {
return;
}
switch (args.get(0).resultType()) {
case REAL_RESULT:
case STRING_RESULT:
ItemResult i = args.get(0).resultType();
if (i == ItemResult.REAL_RESULT || i == ItemResult.STRING_RESULT) {
hybrid_type = ItemResult.REAL_RESULT;
decimals = Math.min(decimals_to_set, NOT_FIXED_DEC);
maxLength = floatLength(decimals);
break;
case INT_RESULT:
/* Here we can keep INT_RESULT */
} else if (i == ItemResult.INT_RESULT) {/* Here we can keep INT_RESULT */
hybrid_type = ItemResult.INT_RESULT;
decimals = 0;
break;
/* fall through */
case DECIMAL_RESULT: {
} else if (i == ItemResult.DECIMAL_RESULT) {
hybrid_type = ItemResult.DECIMAL_RESULT;
decimals_to_set = Math.min(DECIMAL_MAX_SCALE, decimals_to_set);
decimals = Math.min(decimals_to_set, DECIMAL_MAX_SCALE);
break;
}
default:
} else {
assert (false); /* This result type isn't handled */
}
}

View File

@@ -33,9 +33,7 @@ public class ItemFuncDiv extends ItemNumOp {
@Override
public void fixLengthAndDec() {
super.fixLengthAndDec();
switch (hybrid_type) {
case REAL_RESULT: {
// see sql/item_func.cc Item_func_div::fix_length_and_dec()
if (hybrid_type == ItemResult.REAL_RESULT) {
decimals = Math.max(args.get(0).decimals, args.get(1).decimals) + prec_increment;
decimals = Math.min(decimals, NOT_FIXED_DEC);
int tmp = floatLength(decimals);
@@ -45,16 +43,14 @@ public class ItemFuncDiv extends ItemNumOp {
maxLength = args.get(0).maxLength - args.get(1).decimals + decimals;
maxLength = Math.min(maxLength, tmp);
}
break;
}
case INT_RESULT:
} else if (hybrid_type == ItemResult.INT_RESULT) {
hybrid_type = ItemResult.DECIMAL_RESULT;
result_precision();
break;
case DECIMAL_RESULT:
} else if (hybrid_type == ItemResult.DECIMAL_RESULT) {
result_precision();
default:
break;
}
}

View File

@@ -37,20 +37,17 @@ public class ItemFuncCoalesce extends ItemFuncNumhybrid {
public void fixLengthAndDec() {
cached_field_type = MySQLcom.agg_field_type(args, 0, args.size());
hybrid_type = MySQLcom.agg_result_type(args, 0, args.size());
switch (hybrid_type) {
case STRING_RESULT:
break;
case DECIMAL_RESULT:
if (hybrid_type == ItemResult.STRING_RESULT) {
} else if (hybrid_type == ItemResult.DECIMAL_RESULT) {
countDecimalLength();
break;
case REAL_RESULT:
} else if (hybrid_type == ItemResult.REAL_RESULT) {
countRealLength();
break;
case INT_RESULT:
} else if (hybrid_type == ItemResult.INT_RESULT) {
decimals = 0;
break;
case ROW_RESULT:
default:
} else {
assert (false);
}
}

View File

@@ -183,26 +183,22 @@ public abstract class ItemFuncMinMax extends ItemFunc {
}
}
switch (cmp_type) {
case INT_RESULT: {
if (cmp_type == ItemResult.INT_RESULT) {
BigInteger nr = valInt();
if (nullValue)
return null;
return nr.toString();
}
case DECIMAL_RESULT: {
} else if (cmp_type == ItemResult.DECIMAL_RESULT) {
BigDecimal bd = valDecimal();
if (nullValue)
return null;
return bd.toString();
}
case REAL_RESULT: {
} else if (cmp_type == ItemResult.REAL_RESULT) {
BigDecimal nr = valReal();
if (nullValue)
return null; /* purecov: inspected */
return nr.toString();
}
case STRING_RESULT: {
} else if (cmp_type == ItemResult.STRING_RESULT) {
String res = null;
for (int i = 0; i < args.size(); i++) {
if (i == 0)
@@ -219,10 +215,7 @@ public abstract class ItemFuncMinMax extends ItemFunc {
return null;
}
return res;
}
case ROW_RESULT:
default:
// This case should never be chosen
} else {// This case should never be chosen
return null;
}
}
@@ -266,15 +259,15 @@ public abstract class ItemFuncMinMax extends ItemFunc {
return MyTime.check_date(ltime, ltime.isNonZeroDate(), fuzzydate, warnings);
}
switch (fieldType()) {
case MYSQL_TYPE_TIME:
FieldTypes i = fieldType();
if (i == FieldTypes.MYSQL_TYPE_TIME) {
return getDateFromTime(ltime);
case MYSQL_TYPE_DATETIME:
case MYSQL_TYPE_TIMESTAMP:
case MYSQL_TYPE_DATE:
} else if (i == FieldTypes.MYSQL_TYPE_DATETIME || i == FieldTypes.MYSQL_TYPE_TIMESTAMP || i == FieldTypes.MYSQL_TYPE_DATE) {
assert (false); // Should have been processed in "compare_as_dates"
// block.
default:
// block.
return getDateFromNonTemporal(ltime, fuzzydate);
} else {
return getDateFromNonTemporal(ltime, fuzzydate);
}
}
@@ -292,21 +285,20 @@ public abstract class ItemFuncMinMax extends ItemFunc {
return false;
}
switch (fieldType()) {
case MYSQL_TYPE_TIME: {
FieldTypes i = fieldType();
if (i == FieldTypes.MYSQL_TYPE_TIME) {
LongPtr result = new LongPtr(0);
cmp_times(result);
if (nullValue)
return true;
MyTime.TIME_from_longlong_time_packed(ltime, result.get());
return false;
}
case MYSQL_TYPE_DATE:
case MYSQL_TYPE_TIMESTAMP:
case MYSQL_TYPE_DATETIME:
} else if (i == FieldTypes.MYSQL_TYPE_DATE || i == FieldTypes.MYSQL_TYPE_TIMESTAMP || i == FieldTypes.MYSQL_TYPE_DATETIME) {
assert (false); // Should have been processed in "compare_as_dates"
// block.
default:
// block.
return getTimeFromNonTemporal(ltime);
} else {
return getTimeFromNonTemporal(ltime);
}
}

Some files were not shown because too many files have changed in this diff Show More