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/02/13 09:10:02 UTC

[GitHub] [incubator-shardingsphere] KomachiSion opened a new issue #4287: ShardingProxy can't get columnMetadata for PostgreSQL

KomachiSion opened a new issue #4287: ShardingProxy can't get columnMetadata for PostgreSQL
URL: https://github.com/apache/incubator-shardingsphere/issues/4287
 
 
   ## Bug Report
   
   Similar problem with #4182 .
   
   ### Which version of ShardingSphere did you use?
   
   4.0.0 and 5.0.0-RC1-SNAPSHOT
   
   ### Which project did you use? Sharding-JDBC or Sharding-Proxy?
   
   Sharding-Proxy
   
   ### Expected behavior
   
   getColumns interface get right columnMetadata.
   
   ### Actual behavior
   
   #### For 4.0.0 version
   
   Throw exception:
   ```
   line 1:167 no viable alternative at input '(SELECTn.nspname,c.relname,a.attname,a.atttypid,a.attnotnullOR(t.typtype='d'ANDt.typnotnull)ASattnotnull,a.atttypmod,a.attlen,row_number()OVER('
   [ERROR] 16:59:15.575 [pool-4-thread-1] o.a.s.s.f.c.CommandExecutorTask - Exception occur: 
   java.lang.UnsupportedOperationException: Cannot support multiple schemas in one SQL
   	at org.apache.shardingsphere.sql.parser.relation.segment.table.TablesContext.setSchema(TablesContext.java:75)
   	at org.apache.shardingsphere.sql.parser.relation.segment.table.TablesContext.<init>(TablesContext.java:63)
   	at org.apache.shardingsphere.sql.parser.relation.statement.impl.CommonSQLStatementContext.<init>(CommonSQLStatementContext.java:41)
   	at org.apache.shardingsphere.sql.parser.relation.statement.impl.SelectSQLStatementContext.<init>(SelectSQLStatementContext.java:79)
   	at org.apache.shardingsphere.sql.parser.relation.SQLStatementContextFactory.newInstance(SQLStatementContextFactory.java:54)
   	at org.apache.shardingsphere.core.route.router.sharding.ShardingRouter.route(ShardingRouter.java:103)
   	at org.apache.shardingsphere.core.route.PreparedStatementRoutingEngine.route(PreparedStatementRoutingEngine.java:63)
   	at org.apache.shardingsphere.core.PreparedQueryShardingEngine.route(PreparedQueryShardingEngine.java:59)
   	at org.apache.shardingsphere.core.BaseShardingEngine.executeRoute(BaseShardingEngine.java:85)
   	at org.apache.shardingsphere.core.BaseShardingEngine.shard(BaseShardingEngine.java:68)
   	at org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.wrapper.PreparedStatementExecutorWrapper.doShardingRoute(PreparedStatementExecutorWrapper.java:83)
   	at org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.wrapper.PreparedStatementExecutorWrapper.route(PreparedStatementExecutorWrapper.java:69)
   	at org.apache.shardingsphere.shardingproxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine.execute(JDBCDatabaseCommunicationEngine.java:83)
   	at org.apache.shardingsphere.shardingproxy.frontend.postgresql.command.query.binary.bind.PostgreSQLComBindExecutor.execute(PostgreSQLComBindExecutor.java:81)
   	at org.apache.shardingsphere.shardingproxy.frontend.command.CommandExecutorTask.executeCommand(CommandExecutorTask.java:92)
   	at org.apache.shardingsphere.shardingproxy.frontend.command.CommandExecutorTask.run(CommandExecutorTask.java:72)
   	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:748)
   ```
   sql.show don't print.
   
   #### For 5.0.0-RC1-SNAPSHOT
   
   can't return right result.
   
   sql.show is 
   ```
   line 1:167 missing ')' at '('
   line 1:178 mismatched input 'BY' expecting {')', ','}
   [INFO ] 16:54:30.806 [pool-4-thread-1] ShardingSphere-SQL - Rule Type: sharding
   [INFO ] 16:54:30.806 [pool-4-thread-1] ShardingSphere-SQL - Logic SQL: SELECT * FROM (SELECT n.nspname,c.relname,a.attname,a.atttypid,a.attnotnull OR (t.typtype = 'd' AND t.typnotnull) AS attnotnull,a.atttypmod,a.attlen,row_number() OVER (PARTITION BY a.attrelid ORDER BY a.attnum) AS attnum, null as attidentity,pg_catalog.pg_get_expr(def.adbin, def.adrelid) AS adsrc,dsc.description,t.typbasetype,t.typtype  FROM pg_catalog.pg_namespace n  JOIN pg_catalog.pg_class c ON (c.relnamespace = n.oid)  JOIN pg_catalog.pg_attribute a ON (a.attrelid=c.oid)  JOIN pg_catalog.pg_type t ON (a.atttypid = t.oid)  LEFT JOIN pg_catalog.pg_attrdef def ON (a.attrelid=def.adrelid AND a.attnum = def.adnum)  LEFT JOIN pg_catalog.pg_description dsc ON (c.oid=dsc.objoid AND a.attnum = dsc.objsubid)  LEFT JOIN pg_catalog.pg_class dc ON (dc.oid=dsc.classoid AND dc.relname='pg_class')  LEFT JOIN pg_catalog.pg_namespace dn ON (dc.relnamespace=dn.oid AND dn.nspname='pg_catalog')  WHERE c.relkind in ('r','p','v','f','m') and a.attnum > 0 AND NOT a.attisdropped  AND c.relname LIKE E't_order') c WHERE true  AND attname LIKE E'%' ORDER BY nspname,c.relname,attnum 
   [INFO ] 16:54:30.806 [pool-4-thread-1] ShardingSphere-SQL - SQLStatement: SelectSQLStatementContext(super=CommonSQLStatementContext(sqlStatement=org.apache.shardingsphere.sql.parser.sql.statement.dml.SelectStatement@5c510d54, tablesContext=TablesContext(tables=[], schema=Optional.absent())), projectionsContext=ProjectionsContext(startIndex=7, stopIndex=7, distinctRow=false, projections=[ShorthandProjection(owner=Optional.absent())], columnLabels=[]), groupByContext=org.apache.shardingsphere.sql.parser.relation.segment.select.groupby.GroupByContext@72dd9bf7, orderByContext=org.apache.shardingsphere.sql.parser.relation.segment.select.orderby.OrderByContext@7747736b, paginationContext=org.apache.shardingsphere.sql.parser.relation.segment.select.pagination.PaginationContext@6a8c3439, containsSubquery=false)
   [INFO ] 16:54:30.806 [pool-4-thread-1] ShardingSphere-SQL - Actual SQL: ds_0 ::: SELECT * FROM (SELECT n.nspname,c.relname,a.attname,a.atttypid,a.attnotnull OR (t.typtype = 'd' AND t.typnotnull) AS attnotnull,a.atttypmod,a.attlen,row_number() OVER (PARTITION BY a.attrelid ORDER BY a.attnum) AS attnum, null as attidentity,pg_catalog.pg_get_expr(def.adbin, def.adrelid) AS adsrc,dsc.description,t.typbasetype,t.typtype  FROM pg_catalog.pg_namespace n  JOIN pg_catalog.pg_class c ON (c.relnamespace = n.oid)  JOIN pg_catalog.pg_attribute a ON (a.attrelid=c.oid)  JOIN pg_catalog.pg_type t ON (a.atttypid = t.oid)  LEFT JOIN pg_catalog.pg_attrdef def ON (a.attrelid=def.adrelid AND a.attnum = def.adnum)  LEFT JOIN pg_catalog.pg_description dsc ON (c.oid=dsc.objoid AND a.attnum = dsc.objsubid)  LEFT JOIN pg_catalog.pg_class dc ON (dc.oid=dsc.classoid AND dc.relname='pg_class')  LEFT JOIN pg_catalog.pg_namespace dn ON (dc.relnamespace=dn.oid AND dn.nspname='pg_catalog')  WHERE c.relkind in ('r','p','v','f','m') and a.attnum > 0 AND NOT a.attisdropped  AND c.relname LIKE E't_order') c WHERE true  AND attname LIKE E'%' ORDER BY nspname,c.relname,attnum 
   
   ```
   
   ### Reason analyze (If you can)
   
   It seems that `getColumns` will be translate to SQL 
   ```sql
   SELECT * FROM (SELECT n.nspname,c.relname,a.attname,a.atttypid,a.attnotnull OR (t.typtype = 'd' AND t.typnotnull) AS attnotnull,a.atttypmod,a.attlen,row_number() OVER (PARTITION BY a.attrelid ORDER BY a.attnum) AS attnum, null as attidentity,pg_catalog.pg_get_expr(def.adbin, def.adrelid) AS adsrc,dsc.description,t.typbasetype,t.typtype  FROM pg_catalog.pg_namespace n  JOIN pg_catalog.pg_class c ON (c.relnamespace = n.oid)  JOIN pg_catalog.pg_attribute a ON (a.attrelid=c.oid)  JOIN pg_catalog.pg_type t ON (a.atttypid = t.oid)  LEFT JOIN pg_catalog.pg_attrdef def ON (a.attrelid=def.adrelid AND a.attnum = def.adnum)  LEFT JOIN pg_catalog.pg_description dsc ON (c.oid=dsc.objoid AND a.attnum = dsc.objsubid)  LEFT JOIN pg_catalog.pg_class dc ON (dc.oid=dsc.classoid AND dc.relname='pg_class')  LEFT JOIN pg_catalog.pg_namespace dn ON (dc.relnamespace=dn.oid AND dn.nspname='pg_catalog')  WHERE c.relkind in ('r','p','v','f','m') and a.attnum > 0 AND NOT a.attisdropped  AND c.relname LIKE E't_order') c WHERE true  AND attname LIKE E'%' ORDER BY nspname,c.relname,attnum 
   ```
   
   but ShardingSphere can't support this SQL.
   
   ### Steps to reproduce the behavior, such as: SQL to execute, sharding rule configuration, when exception occur etc.
   
   configuration:
   ```yaml
   schemaName: sharding_db
   
   dataSources:
     ds_0:
       url: jdbc:postgresql://127.0.0.1:5432/demo_ds_0?serverTimezone=UTC&useSSL=false
       username: root
       password: 1
       connectionTimeoutMilliseconds: 30000
       idleTimeoutMilliseconds: 60000
       maxLifetimeMilliseconds: 1800000
       maxPoolSize: 50
     ds_1:
       url: jdbc:postgresql://127.0.0.1:5432/demo_ds_1?serverTimezone=UTC&useSSL=false
       username: root
       password: 1
       connectionTimeoutMilliseconds: 30000
       idleTimeoutMilliseconds: 60000
       maxLifetimeMilliseconds: 1800000
       maxPoolSize: 50
   
   shardingRule:
     tables:
       t_order:
         actualDataNodes: ds_${0..1}.t_order_${0..1}
         tableStrategy:
           inline:
             shardingColumn: order_id
             algorithmExpression: t_order_${order_id % 2}
         keyGenerator:
           type: SNOWFLAKE
           column: order_id
       t_order_item:
         actualDataNodes: ds_${0..1}.t_order_item_${0..1}
         tableStrategy:
           inline:
             shardingColumn: order_id
             algorithmExpression: t_order_item_${order_id % 2}
         keyGenerator:
           type: SNOWFLAKE
           column: order_item_id
     bindingTables:
       - t_order,t_order_item
     defaultDatabaseStrategy:
       inline:
         shardingColumn: user_id
         algorithmExpression: ds_${user_id % 2}
     defaultTableStrategy:
       none:
   ```
   
   ### Example codes for reproduce this issue (such as a github link).
   
   ```java
   try (Connection connection = DriverManager.getConnection("jdbc:postgresql://localhost:3307/sharding_db", "root", "root"){
       DatabaseMetaData metaData = connection.getMetaData();
       ResultSet rs = metaData.getColumns("sharding_db", null, "t_order", "%");
   }
   ```
   

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


With regards,
Apache Git Services

[GitHub] [shardingsphere] jingshanglu commented on issue #4287: ShardingProxy can't get columnMetadata for PostgreSQL

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


   @KomachiSion @tuohai666  Subquery only support parsing, but sometimes cannot be run. modify sql to a non-subquery is a solution.


----------------------------------------------------------------
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] tuohai666 edited a comment on issue #4287: ShardingProxy can't get columnMetadata for PostgreSQL

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


   The problem is caused by two parts:
   1. The parse error which was fixed via https://github.com/apache/shardingsphere/pull/6045 .
   2. The SQL contains subquery which will be fixed via https://github.com/apache/shardingsphere/issues/3571 in future .
   
   For now, this issue is partial fixed. Another partial will be fixed together with https://github.com/apache/shardingsphere/issues/3571.


----------------------------------------------------------------
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] tuohai666 commented on issue #4287: ShardingProxy can't get columnMetadata for PostgreSQL

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


   The problem is caused by two parts:
   1. The pars error which was fixed via https://github.com/apache/shardingsphere/pull/6045 .
   2. The SQL contains subquery which will be fixed via https://github.com/apache/shardingsphere/issues/4696 in future .
   
   For now, this issue is partial fixed. Another partial will be fixed together with https://github.com/apache/shardingsphere/issues/4696.


----------------------------------------------------------------
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] tuohai666 closed issue #4287: ShardingProxy can't get columnMetadata for PostgreSQL

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


   


----------------------------------------------------------------
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] tuohai666 edited a comment on issue #4287: ShardingProxy can't get columnMetadata for PostgreSQL

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


   The problem is caused by two parts:
   1. The parse error which was fixed via https://github.com/apache/shardingsphere/pull/6045 .
   2. The SQL contains subquery which will be fixed via https://github.com/apache/shardingsphere/issues/4696 in future .
   
   For now, this issue is partial fixed. Another partial will be fixed together with https://github.com/apache/shardingsphere/issues/4696.


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