You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2020/08/19 10:20:52 UTC

[GitHub] [shardingsphere] sandynz opened a new issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

sandynz opened a new issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939


   ## Bug Report
   
   ### Which version of ShardingSphere did you use?
   5.0.0-RC1-SNAPSHOT
   
   ### Which project did you use? ShardingSphere-JDBC or ShardingSphere-Proxy?
   ShardingSphere-Proxy
   
   ### Expected behavior
   Sql could be executed correctly, just like MySQL 5.7 server.
   
   ### Actual behavior
   Exception thrown:
   ```
   Exception in thread "main" java.sql.SQLException: Parameter index out of bounds. 4 is not between valid values of 1 and 3
   	at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:965)
   	at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:898)
   	at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:887)
   	at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:861)
   	at com.mysql.jdbc.ServerPreparedStatement.getBinding(ServerPreparedStatement.java:867)
   	at com.mysql.jdbc.ServerPreparedStatement.setInt(ServerPreparedStatement.java:1856)
   ```
   
   ### Reason analyze (If you can)
   ```
   mysql-connector-java-5.1.48
   ServerPreparedStatement.java, serverPrepare(String sql)
                   Buffer prepareResultPacket = mysql.sendCommand(MysqlDefs.COM_PREPARE, sql, null, false, characterEncoding, 0);
   
                   if (this.connection.versionMeetsMinimum(4, 1, 1)) {
                       // 4.1.1 and newer use the first byte as an 'ok' or 'error' flag, so move the buffer pointer past it to start reading the statement id.
                       prepareResultPacket.setPosition(1);
                   } else {
                       // 4.1.0 doesn't use the first byte as an 'ok' or 'error' flag
                       prepareResultPacket.setPosition(0);
                   }
   
                   this.serverStatementId = prepareResultPacket.readLong();
                   this.fieldCount = prepareResultPacket.readInt();
                   this.parameterCount = prepareResultPacket.readInt();
                   this.parameterBindings = new BindValue[this.parameterCount];
   this.parameterCount is 3, which is from proxy server side, but it's 4 in fact
   ```
   
   ```
   MySQLComStmtPrepareExecutor.java
           SQLStatement sqlStatement = schema.getRuntimeContext().getSqlParserEngine().parse(packet.getSql(), true);
           if (!MySQLComStmtPrepareChecker.isStatementAllowed(sqlStatement)) {
               result.add(new MySQLErrPacket(++currentSequenceId, MySQLServerErrorCode.ER_UNSUPPORTED_PS));
               return result;
           }
           int parametersCount = sqlStatement.getParameterCount();
   parametersCount is 3 after parsing
   ```
   
   ### Steps to reproduce the behavior, such as: SQL to execute, sharding rule configuration, when exception occur etc.
   Run sql via jdbc:
   ```
   update t_order set status = ? where order_id = ? or (user_id = ? and status='PAID') limit ?
   
   delete from t_order where order_id = ? or (user_id = ? and status = ?) limit ?
   ```
   
   ### Example codes for reproduce this issue (such as a github link).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero closed issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
tristaZero closed issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] sandynz commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
sandynz commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-677356028


   > 
   > 
   > @sandynz Hi, I have used the master branch to test this exception, but it cat not be reproduced. Can you try it with the latest code of the master branch?
   
   Hi @strongduanmu , I tested it on lastest master branch again, commit: bfb1486549e9908f8784210fe2100f1c5fdd212d (2020-08-19), run `org.apache.shardingsphere.proxy.Bootstrap`, it doesn't work.
   
   My config-sharding.yaml
   ```
   schemaName: sharding_db
   
   dataSourceCommon:
     username: root
     password: test
     connectionTimeoutMilliseconds: 30000
     idleTimeoutMilliseconds: 60000
     maxLifetimeMilliseconds: 1800000
     maxPoolSize: 50
     minPoolSize: 1
     maintenanceIntervalMilliseconds: 30000
   
   dataSources:
     ds:
       url: jdbc:mysql://localhost:3306/demo_ds_1?serverTimezone=UTC&useSSL=false&useUnicode=true&characterEncoding=UTF-8
   
   rules:
     - !SHARDING
       tables:
         t_order:
           actualDataNodes: ds.t_order_${0..1}
           tableStrategy:
             standard:
               shardingColumn: order_id
               shardingAlgorithmName: t_order_inline
           keyGenerateStrategy:
             column: order_id
             keyGeneratorName: snowflake
         t_order_item:
           actualDataNodes: ds.t_order_item_${0..1}
           tableStrategy:
             standard:
               shardingColumn: order_id
               shardingAlgorithmName: t_order_item_inline
           keyGenerateStrategy:
             column: order_item_id
             keyGeneratorName: snowflake
       bindingTables:
         - t_order,t_order_item
       broadcastTables:
         - t_address
   
       shardingAlgorithms:
         t_order_inline:
           type: INLINE
           props:
             algorithm.expression: t_order_${order_id % 2}
         t_order_item_inline:
           type: INLINE
           props:
             algorithm.expression: t_order_item_${order_id % 2}
   
       keyGenerators:
         snowflake:
           type: SNOWFLAKE
           props:
             worker.id: 123
   ```
   
   My database:
   ```
   mysql> use demo_ds_1
   Database changed
   
   CREATE TABLE IF NOT EXISTS t_address (address_id BIGINT NOT NULL, address_name VARCHAR(100) NOT NULL, PRIMARY KEY (address_id));
   CREATE TABLE IF NOT EXISTS `t_order_0` (`order_id` INT NOT NULL, `user_id` INT NOT NULL, `status` VARCHAR(45) NULL, PRIMARY KEY (`order_id`));
   CREATE TABLE IF NOT EXISTS `t_order_1` (`order_id` INT NOT NULL, `user_id` INT NOT NULL, `status` VARCHAR(45) NULL, PRIMARY KEY (`order_id`));
   CREATE TABLE IF NOT EXISTS `t_order_item_0` (`order_item_id` INT NOT NULL, `order_id` INT NOT NULL, `user_id` INT NOT NULL, `status` VARCHAR(45) NULL, PRIMARY KEY (`order_item_id`));
   CREATE TABLE IF NOT EXISTS `t_order_item_1` (`order_item_id` INT NOT NULL, `order_id` INT NOT NULL, `user_id` INT NOT NULL, `status` VARCHAR(45) NULL, PRIMARY KEY (`order_item_id`));
   
   mysql> show tables;
   +---------------------+
   | Tables_in_demo_ds_1 |
   +---------------------+
   | t_address           |
   | t_order_0           |
   | t_order_1           |
   | t_order_item_0      |
   | t_order_item_1      |
   +---------------------+
   5 rows in set (0.00 sec)
   ```
   
   My client test code snippet:
   ```
   public static void main(String[] args) throws Exception {
   	final boolean useServerPrepStmts = true;
   	try (Connection connection = DriverManager.getConnection("jdbc:mysql://127.0.0.1:3307/sharding_db?serverTimezone=UTC&useSSL=false" + (useServerPrepStmts ? "&useServerPrepStmts=true&cachePrepStmts=true" : ""), "root", "root")) {
   		int orderId = 1, userId = 2;
   		String status = "PAID";
   		try (PreparedStatement statement = connection.prepareStatement("update t_order set status = ? where order_id = ? or (user_id = ? and status='PAID') limit ?")) {
   			statement.setString(1, "SENT");
   			statement.setLong(2, orderId);
   			statement.setLong(3, userId);
   			statement.setInt(4, 1);
   			int updateCount = statement.executeUpdate();
   			LOG.info("update count={}", updateCount);
   		}
   	}
   }
   ```
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-678014284


   @strongduanmu Very appreciated your initiative. Moreover, please involves more in **Our** community. :-) 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] strongduanmu commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-677978145


   @sandynz Now the `update statement` and `delete statement` do not support limit clause parameterized parse, and I will try to implement this feature. Thanks again for your feedback. 😀


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] strongduanmu commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-676887222


   @sandynz Hi, I have used the master branch to test this exception, but it cat not be reproduced. Can you try it with the latest code of the master branch?
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] sandynz commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
sandynz commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-678880876


   > 
   > 
   > @sandynz Hi, after this [PR](https://github.com/apache/shardingsphere/pull/6997) is merged into the master branch, you can try again. It should be noted that the update limit and delete limit statements do not support sharding route to multiple data nodes, because its result are incorrect, and currently only support sharding route to single data node.
   
   Hi @strongduanmu , thanks.
   I've tested it again:
   - `update t_order set status = ? where order_id = ? or (user_id = ? and status='PAID') limit ?`, failed, notice `UPDATE ... LIMIT can not support sharding route to multiple data nodes.`
   - `update t_order set status = ? where order_id = ? limit ?`, works
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-676910313


   @strongduanmu Nice job!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-678879779


   @sandynz Hi, Thanks to @strongduanmu 's work, this issue was fixed conditionally. If possible, welcome your test.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
tristaZero commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-678931019


   @sandynz Thanks for your rapid tests!
   That is precisely in our expectations. :)
   What do you think @strongduanmu ?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] tristaZero edited a comment on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-678014284


   Very appreciated your initiative. Moreover, please involves more in **OUR** community. :-) 
   So the assignee @strongduanmu of this issue showed up.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] strongduanmu commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-678649124


   @sandynz Hi, after this [PR](https://github.com/apache/shardingsphere/pull/6997) is merged into the master branch, you can try again. It should be noted that the update limit and delete limit statements do not support sharding route to multiple data nodes, because its result are incorrect, and currently only support sharding route to single data node.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] strongduanmu commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-677455470


   @sandynz I will test it again!
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [shardingsphere] strongduanmu commented on issue #6939: Parameterized limit in update/delete statement doesn't work on MySQL proxy

Posted by GitBox <gi...@apache.org>.
strongduanmu commented on issue #6939:
URL: https://github.com/apache/shardingsphere/issues/6939#issuecomment-679060707


   > @sandynz Thanks for your rapid tests!
   > That is precisely in our expectations. :)
   > What do you think @strongduanmu ?
   
   @tristaZero Yes, currently only support sharding route to single data node.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org