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 2022/09/16 07:51:55 UTC

[GitHub] [shardingsphere] peilinqian opened a new issue, #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

peilinqian opened a new issue, #21013:
URL: https://github.com/apache/shardingsphere/issues/21013

   ## Bug Report
   
   **For English only**, other languages will not accept.
   
   Before report a bug, make sure you have:
   
   - Searched open and closed [GitHub issues](https://github.com/apache/shardingsphere/issues).
   - Read documentation: [ShardingSphere Doc](https://shardingsphere.apache.org/document/current/en/overview).
   
   Please pay attention on issues you submitted, because we maybe need more details. 
   If no response anymore and we cannot reproduce it on current information, we will **close it**.
   
   Please answer these questions before submitting your issue. Thanks!
   
   ### Which version of ShardingSphere did you use?
   
   ### Which project did you use? ShardingSphere-JDBC or ShardingSphere-Proxy?
   
   ### Expected behavior
   After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, the next SQL statement is executed normally
   
   ### Actual behavior
   After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed
   
   ### Reason analyze (If you can)
   
   ### Steps to reproduce the behavior, such as: SQL to execute, sharding rule configuration, when exception occur etc.
   ```
   drop view if exists t_ssdb_tb_view;
   drop view if exists t_ssdb_tb_view_new;
   drop table if exists t_ssdb_tb;
   create table t_ssdb_tb(id int,c_id int,a int,b char(10));
   insert into t_ssdb_tb values (1,1,10,'test11'),(1,2,10,'Test12'),(1,2,null,'test12'),
   (2,1,20,'test21'),(2,2,20,'test22'),(2,3,null,'Test22'),
   (3,3,null,'test3'),(3,3,null,'test3'),(4,4,40,'test4'),(5,5,50,'Test5');
   show sharding binding table rules;
   create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);
   select pg_sleep(2);
   ```
   
   ```
   drop view if exists t_ssdb_tb_view;
   DROP VIEW
   drop view if exists t_ssdb_tb_view_new;
   DROP VIEW
   drop table if exists t_ssdb_tb;
   DROP TABLE
   create table t_ssdb_tb(id int,c_id int,a int,b char(10));
   CREATE TABLE
   insert into t_ssdb_tb values (1,1,10,'test11'),(1,2,10,'Test12'),(1,2,null,'test12'),
   (2,1,20,'test21'),(2,2,20,'test22'),(2,3,null,'Test22'),
   (3,3,null,'test3'),(3,3,null,'test3'),(4,4,40,'test4'),(5,5,50,'Test5');
   INSERT 0 10
   
   show sharding binding table rules;
    sharding_binding_tables
   -------------------------
   (0 rows)
   
   create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);
   SUCCESS
   select pg_sleep(2);
   ERROR:  java.lang.NullPointerException
   ```
   
   background log
   
   ```
   [INFO ] 2022-09-16 15:39:04.601 [Connection-3-ThreadExecutor] ShardingSphere-SQL - Logic SQL: select pg_sleep(2);
   [INFO ] 2022-09-16 15:39:04.601 [Connection-3-ThreadExecutor] ShardingSphere-SQL - SQLStatement: OpenGaussSelectStatement(super=SelectStatement(super=AbstractSQLStatement(parameterCount=0, parameterMarkerSegments=[], commentSegments=[]), projections=ProjectionsSegment(startIndex=7, stopIndex=17, projections=[ExpressionProjectionSegment(startIndex=7, stopIndex=17, text=pg_sleep(2), expr=FunctionSegment(startIndex=7, stopIndex=17, functionName=pg_sleep, parameters=[LiteralExpressionSegment(startIndex=16, stopIndex=16, literals=2)], text=pg_sleep(2), owner=null), alias=Optional.empty)], distinctRow=false), from=null, where=Optional.empty, groupBy=Optional.empty, having=Optional.empty, orderBy=Optional.empty, combines=[]), limit=Optional.empty, lock=Optional.empty, window=Optional.empty)
   [INFO ] 2022-09-16 15:39:04.601 [Connection-3-ThreadExecutor] ShardingSphere-SQL - Actual SQL: ds1_1 ::: select pg_sleep(2);
   [ERROR] 2022-09-16 15:39:04.601 [Connection-3-ThreadExecutor] o.a.s.p.f.c.CommandExecutorTask - Exception occur:
   java.lang.NullPointerException: null
           at org.apache.shardingsphere.transaction.xa.XAShardingSphereTransactionManager.isInTransaction(XAShardingSphereTransactionManager.java:73)
           at org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCBackendDataSource.isInTransaction(JDBCBackendDataSource.java:118)
           at org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCBackendDataSource.createConnection(JDBCBackendDataSource.java:109)
           at org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCBackendDataSource.getConnections(JDBCBackendDataSource.java:80)
           at org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.JDBCBackendDataSource.getConnections(JDBCBackendDataSource.java:53)
           at org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection.createNewConnections(JDBCBackendConnection.java:96)
           at org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection.getConnections(JDBCBackendConnection.java:87)
           at org.apache.shardingsphere.infra.executor.sql.prepare.driver.DriverExecutionPrepareEngine.group(DriverExecutionPrepareEngine.java:88)
           at org.apache.shardingsphere.infra.executor.sql.prepare.AbstractExecutionPrepareEngine.prepare(AbstractExecutionPrepareEngine.java:62)
           at org.apache.shardingsphere.proxy.backend.communication.ProxySQLExecutor.useDriverToExecute(ProxySQLExecutor.java:187)
           at org.apache.shardingsphere.proxy.backend.communication.ProxySQLExecutor.execute(ProxySQLExecutor.java:153)
           at org.apache.shardingsphere.proxy.backend.communication.ProxySQLExecutor.execute(ProxySQLExecutor.java:147)
           at org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine.execute(JDBCDatabaseCommunicationEngine.java:130)
           at org.apache.shardingsphere.proxy.backend.handler.data.impl.UnicastDatabaseBackendHandler.execute(UnicastDatabaseBackendHandler.java:76)
           at org.apache.shardingsphere.proxy.frontend.opengauss.command.query.simple.OpenGaussComQueryExecutor.execute(OpenGaussComQueryExecutor.java:77)
           at org.apache.shardingsphere.proxy.frontend.command.CommandExecutorTask.executeCommand(CommandExecutorTask.java:111)
           at org.apache.shardingsphere.proxy.frontend.command.CommandExecutorTask.run(CommandExecutorTask.java:78)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:750)
   ```
   
   
   ### 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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org.apache.org

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


[GitHub] [shardingsphere] peilinqian commented on issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

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

   test_db=> create table t_broadcast(id int,c_id int,a int,b char(10));
   CREATE TABLE
   test_db=> insert into t_broadcast values (1,1,10,'test11'),(1,2,10,'Test12'),(1,2,null,'test12'),
   test_db-> (2,1,20,'test21'),(2,2,20,'test22'),(2,3,null,'Test22'),
   test_db-> (3,3,null,'test3'),(3,3,null,'test3'),(4,4,40,'test4'),(5,5,50,'');
   INSERT 0 10
   test_db=>
   test_db=> --创建视图;expect:成功
   test_db=> alter sharding broadcast table rules (t_broadcast,t_broadcast_view);
   SUCCESS
   test_db=> create view t_broadcast_view as select * from t_broadcast;
   CREATE VIEW
   test_db=> select * from t_broadcast_view where id=4;
    id | c_id | a  |     b
   ----+------+----+------------
     4 |    4 | 40 | test4
   (1 row)
   
   test_db=>
   test_db=> --增加修改改后名称的配置规则,改视图名称,修改后的名称配置广播规则;expect:成功
   test_db=> drop sharding broadcast table rules (t_broadcast,t_broadcast_view,t_broadcast_view_new);
   SUCCESS
   test_db=> create sharding broadcast table rules (t_broadcast,t_broadcast_view,t_broadcast_view_new);
   SUCCESS
   test_db=> select pg_sleep(2);
    pg_sleep
   ----------
   
   (1 row)
   
   test_db=> alter view t_broadcast_view rename to t_broadcast_view_new;
   ERROR:  String index out of range: -1
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] tuichenchuxin closed issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

Posted by GitBox <gi...@apache.org>.
tuichenchuxin closed issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed
URL: https://github.com/apache/shardingsphere/issues/21013


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] peilinqian commented on issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

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

   ```
   rules:
   - !SHARDING
     tables:
       lineitem:
         actualDataNodes: ds_${0..1}.lineitem_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: l_partkey
             shardingAlgorithmName: lineitem-database-inline
         tableStrategy:
           standard:
             shardingColumn: l_orderkey
             shardingAlgorithmName: lineitem-table-inline
       customer:
         actualDataNodes: ds_${0..1}.customer_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: c_custkey
             shardingAlgorithmName: customer-database-inline
         tableStrategy:
           standard:
             shardingColumn: c_nationkey
             shardingAlgorithmName: customer-table-inline
       nation:
         actualDataNodes: ds_${0..1}.nation_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: n_nationkey
             shardingAlgorithmName: nation-database-inline
         tableStrategy:
           standard:
             shardingColumn: n_regionkey
             shardingAlgorithmName: nation-table-inline
       orders:
         actualDataNodes: ds_${0..1}.orders_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: o_orderkey
             shardingAlgorithmName: orders-database-inline
         tableStrategy:
           standard:
             shardingColumn: o_custkey
             shardingAlgorithmName: orders-table-inline
       part:
         actualDataNodes: ds_${0..1}.part_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: p_partkey
             shardingAlgorithmName: part-database-inline
         tableStrategy:
           standard:
             shardingColumn: p_size
             shardingAlgorithmName: part-table-inline
       partsupp:
         actualDataNodes: ds_${0..1}.partsupp_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: ps_partkey
             shardingAlgorithmName: partsupp-database-inline
         tableStrategy:
           standard:
             shardingColumn: ps_suppkey
             shardingAlgorithmName: partsupp-table-inline
       region:
         actualDataNodes: ds_${0..1}.region
         databaseStrategy:
           standard:
             shardingColumn: r_regionkey
             shardingAlgorithmName: region-database-inline
       supplier:
         actualDataNodes: ds_${0..1}.supplier_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: s_suppkey
             shardingAlgorithmName: supplier-database-inline
         tableStrategy:
           standard:
             shardingColumn: s_nationkey
             shardingAlgorithmName: supplier-table-inline
     defaultDatabaseStrategy:
       none:
     defaultTableStrategy:
       none:
   
     shardingAlgorithms:
       lineitem-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${l_partkey % 2}
       lineitem-table-inline:
         type: INLINE
         props:
           algorithm-expression: lineitem_${l_orderkey % 2}
       customer-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${c_custkey % 2}
       customer-table-inline:
         type: INLINE
         props:
           algorithm-expression: customer_${c_nationkey % 2}
       nation-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${n_nationkey % 2}
       nation-table-inline:
         type: INLINE
         props:
           algorithm-expression: nation_${n_regionkey % 2}
       orders-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${o_orderkey % 2}
       orders-table-inline:
         type: INLINE
         props:
           algorithm-expression: orders_${o_custkey % 2}
       part-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${p_partkey % 2}
       part-table-inline:
         type: INLINE
         props:
           algorithm-expression: part_${p_size % 2}
       partsupp-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${ps_partkey % 2}
       partsupp-table-inline:
         type: INLINE
         props:
           algorithm-expression: partsupp_${ps_suppkey % 2}
       region-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${r_regionkey % 2}
       supplier-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${s_suppkey % 2}
       supplier-table-inline:
         type: INLINE
         props:
           algorithm-expression: supplier_${s_nationkey % 2}
   
   - !READWRITE_SPLITTING
     dataSources:
       ds_0:
         staticStrategy:
           writeDataSourceName: write_0
           readDataSourceNames:
             - read0_0
             - write_0
         loadBalancerName: roundRobin
       ds_1:
         staticStrategy:
           writeDataSourceName: write_1
           readDataSourceNames:
             - read1_0
             - write_1
         loadBalancerName: roundRobin
     loadBalancers:
       roundRobin:
         type: ROUND_ROBIN
   ```


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] FlyingZC commented on issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

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

   <img width="489" alt="image" src="https://user-images.githubusercontent.com/19788130/190595597-b32d0989-cf8a-4493-b536-48b8d27d14ad.png">
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] peilinqian commented on issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

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

   rules:
   - !SHARDING
     tables:
       lineitem:
         actualDataNodes: ds_${0..1}.lineitem_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: l_partkey
             shardingAlgorithmName: lineitem-database-inline
         tableStrategy:
           standard:
             shardingColumn: l_orderkey
             shardingAlgorithmName: lineitem-table-inline
       customer:
         actualDataNodes: ds_${0..1}.customer_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: c_custkey
             shardingAlgorithmName: customer-database-inline
         tableStrategy:
           standard:
             shardingColumn: c_nationkey
             shardingAlgorithmName: customer-table-inline
       nation:
         actualDataNodes: ds_${0..1}.nation_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: n_nationkey
             shardingAlgorithmName: nation-database-inline
         tableStrategy:
           standard:
             shardingColumn: n_regionkey
             shardingAlgorithmName: nation-table-inline
       orders:
         actualDataNodes: ds_${0..1}.orders_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: o_orderkey
             shardingAlgorithmName: orders-database-inline
         tableStrategy:
           standard:
             shardingColumn: o_custkey
             shardingAlgorithmName: orders-table-inline
       part:
         actualDataNodes: ds_${0..1}.part_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: p_partkey
             shardingAlgorithmName: part-database-inline
         tableStrategy:
           standard:
             shardingColumn: p_size
             shardingAlgorithmName: part-table-inline
       partsupp:
         actualDataNodes: ds_${0..1}.partsupp_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: ps_partkey
             shardingAlgorithmName: partsupp-database-inline
         tableStrategy:
           standard:
             shardingColumn: ps_suppkey
             shardingAlgorithmName: partsupp-table-inline
       region:
         actualDataNodes: ds_${0..1}.region
         databaseStrategy:
           standard:
             shardingColumn: r_regionkey
             shardingAlgorithmName: region-database-inline
       supplier:
         actualDataNodes: ds_${0..1}.supplier_${0..1}
         databaseStrategy:
           standard:
             shardingColumn: s_suppkey
             shardingAlgorithmName: supplier-database-inline
         tableStrategy:
           standard:
             shardingColumn: s_nationkey
             shardingAlgorithmName: supplier-table-inline
     defaultDatabaseStrategy:
       none:
     defaultTableStrategy:
       none:
   
     shardingAlgorithms:
       lineitem-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${l_partkey % 2}
       lineitem-table-inline:
         type: INLINE
         props:
           algorithm-expression: lineitem_${l_orderkey % 2}
       customer-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${c_custkey % 2}
       customer-table-inline:
         type: INLINE
         props:
           algorithm-expression: customer_${c_nationkey % 2}
       nation-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${n_nationkey % 2}
       nation-table-inline:
         type: INLINE
         props:
           algorithm-expression: nation_${n_regionkey % 2}
       orders-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${o_orderkey % 2}
       orders-table-inline:
         type: INLINE
         props:
           algorithm-expression: orders_${o_custkey % 2}
       part-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${p_partkey % 2}
       part-table-inline:
         type: INLINE
         props:
           algorithm-expression: part_${p_size % 2}
       partsupp-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${ps_partkey % 2}
       partsupp-table-inline:
         type: INLINE
         props:
           algorithm-expression: partsupp_${ps_suppkey % 2}
       region-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${r_regionkey % 2}
       supplier-database-inline:
         type: INLINE
         props:
           algorithm-expression: ds_${s_suppkey % 2}
       supplier-table-inline:
         type: INLINE
         props:
           algorithm-expression: supplier_${s_nationkey % 2}


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] peilinqian commented on issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

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

   server.yaml
   ```
   mode:
     type: Cluster
     repository:
       type: ZooKeeper
       props:
         namespace: governance_ds
         server-lists: localhost:2181
         retryIntervalMilliseconds: 5000000
         timeToLiveSeconds: 6000000
         maxRetries: 3
         operationTimeoutMilliseconds: 5000000
     overwrite: true
   
   rules:
     - !AUTHORITY
       users:
         - root@%:root
         - sharding@:sharding
       provider:
         type: ALL_PERMITTED
     - !TRANSACTION
       defaultType: XA
       #providerType: Atomikos
       providerType: Narayana
   #  - !SQL_PARSER
   #    sqlCommentParseEnabled: true
   #    sqlStatementCache:
   #      initialCapacity: 2000
   #      maximumSize: 65535
   #    parseTreeCache:
   #      initialCapacity: 128
   #      maximumSize: 1024
   ```
   
   
   
   ```
   schemaName: test_db
   dataSources:
     ds1_0:
       connectionTimeoutMilliseconds: 30000
       idleTimeoutMilliseconds: 60000
       maxLifetimeMilliseconds: 1800000
       maxPoolSize: 260
       minPoolSize: 1
       password: Test@123
       url: jdbc:opengauss://90.90.44.171:14000/test_db?batchMode=on
       username: tpccuser
     ds1_1:
       connectionTimeoutMilliseconds: 30000
       idleTimeoutMilliseconds: 60000
       maxLifetimeMilliseconds: 1800000
       maxPoolSize: 260
       minPoolSize: 1
       password: Test@123
       url: jdbc:opengauss://90.90.44.171:15000/test_db?batchMode=on
       username: tpccuser
   rules:
   - !SHARDING
     tables:
       t_single:
         actualDataNodes: ds1_0.t_single
       t_single_view:
         actualDataNodes: ds1_0.t_single_view
       t_single_view_new:
         actualDataNodes: ds1_0.t_single_view_new
       t_ssdb_tb:
         actualDataNodes: ds1_${0..1}.t_ssdb_tb_${0..1}
         databaseStrategy:
           standard:
             shardingAlgorithmName: ds1_database_inline
             shardingColumn: id
         tableStrategy:
           standard:
             shardingColumn: c_id
             shardingAlgorithmName: ts_t_ssdb_tb_inline
       t_ssdb_tb_view:
         actualDataNodes: ds1_${0..1}.t_ssdb_tb_view_${0..1}
         databaseStrategy:
           standard:
             shardingAlgorithmName: ds1_database_inline
             shardingColumn: id
         tableStrategy:
           standard:
             shardingColumn: c_id
             shardingAlgorithmName: ts_t_ssdb_tb_view_inline
       t_ssdb:
         actualDataNodes: ds1_${0..1}.t_ssdb
         databaseStrategy:
           standard:
             shardingAlgorithmName: ds1_database_inline
             shardingColumn: id
       t_ssdb_view:
         actualDataNodes: ds1_${0..1}.t_ssdb_view
         databaseStrategy:
           standard:
             shardingAlgorithmName: ds1_database_inline
             shardingColumn: id
       t_ssdb_tb1:
         actualDataNodes: ds1_${0..1}.t_ssdb_tb1_${0..1}
         databaseStrategy:
           standard:
             shardingAlgorithmName: ds1_database_inline
             shardingColumn: id
         tableStrategy:
           standard:
             shardingColumn: c_id
             shardingAlgorithmName: ts_t_ssdb_tb1_inline
       t_ssdb_tb2:
         actualDataNodes: ds1_${0..1}.t_ssdb_tb2_${0..1}
         databaseStrategy:
           standard:
             shardingAlgorithmName: ds_t_ssdb_tb2_inline
             shardingColumn: id
         tableStrategy:
           standard:
             shardingColumn: c_id
             shardingAlgorithmName: ts_t_ssdb_tb2_inline
     broadcastTables:
       - t_broadcast
       - t_broadcast_view
     bindingTables:
       - t_ssdb_tb, t_ssdb_tb_view, t_ssdb_tb1
     defaultTableStrategy:
       none:
     defaultDatabaseStrategy:
       none:
     shardingAlgorithms:
       ds1_database_inline:
         type: INLINE
         props:
           algorithm-expression: ds1_${id % 2}
           allow-range-query-with-inline-sharding: true
       ts_t_ssdb_tb_inline:
         type: INLINE
         props:
           algorithm-expression: t_ssdb_tb_${c_id % 2}
           allow-range-query-with-inline-sharding: true
       ts_t_ssdb_tb1_inline:
         type: INLINE
         props:
           algorithm-expression: t_ssdb_tb1_${c_id % 2}
           allow-range-query-with-inline-sharding: true
       ts_t_ssdb_tb_view_inline:
         type: INLINE
         props:
           algorithm-expression: t_ssdb_tb_view_${c_id % 2}
           allow-range-query-with-inline-sharding: true
       ds_t_ssdb_tb2_inline:
         type: INLINE
         props:
           algorithm-expression: ds1_${id % 2}
       ts_t_ssdb_tb2_inline:
         type: INLINE
         props:
           algorithm-expression: t_ssdb_tb2_${c_id % 2}
           allow-range-query-with-inline-sharding: true
   
   ```
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] FlyingZC commented on issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

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

   Sorry, I can't reproduce the bug,version: f1cab653542d80b9a2b7a5cb4d39c3277862ecda, cluster mode. @peilinqian 
   <img width="808" alt="image" src="https://user-images.githubusercontent.com/19788130/190937624-3400a084-afee-4653-8932-b59a4880415c.png">
   <img width="1072" alt="image" src="https://user-images.githubusercontent.com/19788130/190937687-191d3e29-d153-47dc-bdce-750daa01911e.png">
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] FlyingZC commented on issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

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

   <img width="823" alt="image" src="https://user-images.githubusercontent.com/19788130/190945744-2d2d07ca-f0a9-4be8-97ec-15fdc99b62d8.png">
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] FlyingZC commented on issue #21013: After executing the "create sharding binding table rules (t_ssdb_tb_view, t_ssdb_tb);" statement, an error will be reported when the next SQL statement is executed

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

   Will this operation cause the data source to be closed, which will cause the transaction manager to be closed. @zhaojinchao95 


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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