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/04/24 18:00:45 UTC

[GitHub] [shardingsphere] TaoZhiMLND opened a new issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

TaoZhiMLND opened a new issue #5309:
URL: https://github.com/apache/shardingsphere/issues/5309


   The Junit test in the transaction-2pc-xa-raw-jdbc sample module failed when I used oracle database 12c and ojbdc8.jar. 
   The exception stack is as follows, 
   ```
   java.sql.SQLSyntaxErrorException: ORA-00942: 表或视图不存在
   
   
   	at oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:509)
   	at oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:461)
   	at oracle.jdbc.driver.T4C8Oall.processError(T4C8Oall.java:1104)
   	at oracle.jdbc.driver.T4CTTIfun.receive(T4CTTIfun.java:550)
   	at oracle.jdbc.driver.T4CTTIfun.doRPC(T4CTTIfun.java:268)
   	at oracle.jdbc.driver.T4C8Oall.doOALL(T4C8Oall.java:655)
   	at oracle.jdbc.driver.T4CStatement.doOall8(T4CStatement.java:229)
   	at oracle.jdbc.driver.T4CStatement.doOall8(T4CStatement.java:41)
   	at oracle.jdbc.driver.T4CStatement.executeForRows(T4CStatement.java:928)
   	at oracle.jdbc.driver.OracleStatement.doExecuteWithTimeout(OracleStatement.java:1205)
   	at oracle.jdbc.driver.OracleStatement.executeUpdateInternal(OracleStatement.java:1747)
   	at oracle.jdbc.driver.OracleStatement.executeLargeUpdate(OracleStatement.java:1712)
   	at oracle.jdbc.driver.OracleStatement.executeUpdate(OracleStatement.java:1699)
   	at oracle.jdbc.driver.OracleStatementWrapper.executeUpdate(OracleStatementWrapper.java:285)
   	at oracle.jdbc.OracleDatabaseMetaData.getIndexInfo(OracleDatabaseMetaData.java:3802)
   	at org.apache.shardingsphere.sql.parser.binder.metadata.index.IndexMetaDataLoader.load(IndexMetaDataLoader.java:50)
   	at org.apache.shardingsphere.sql.parser.binder.metadata.table.TableMetaDataLoader.load(TableMetaDataLoader.java:46)
   	at org.apache.shardingsphere.core.metadata.ShardingMetaDataLoader.load(ShardingMetaDataLoader.java:84)
   	at org.apache.shardingsphere.core.metadata.ShardingMetaDataLoader.load(ShardingMetaDataLoader.java:67)
   	at org.apache.shardingsphere.core.metadata.ShardingMetaDataLoader.load(ShardingMetaDataLoader.java:53)
   	at org.apache.shardingsphere.underlying.common.metadata.schema.RuleSchemaMetaDataLoader.load(RuleSchemaMetaDataLoader.java:70)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.core.context.AbstractRuntimeContext.createMetaData(AbstractRuntimeContext.java:91)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.core.context.AbstractRuntimeContext.<init>(AbstractRuntimeContext.java:77)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.core.context.impl.ShardingRuntimeContext.<init>(ShardingRuntimeContext.java:48)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.core.datasource.ShardingDataSource.<init>(ShardingDataSource.java:44)
   	at org.apache.shardingsphere.shardingjdbc.api.ShardingDataSourceFactory.createDataSource(ShardingDataSourceFactory.java:48)
   	at org.apache.shardingsphere.shardingjdbc.api.yaml.YamlShardingDataSourceFactory.createDataSource(YamlShardingDataSourceFactory.java:50)
   	at org.apache.shardingsphere.example.transaction.xa.raw.jdbc.XAOrderService.<init>(XAOrderService.java:42)
   	at org.apache.shardingsphere.example.transaction.xa.raw.jdbc.XAOrderServiceTest.setUp(XAOrderServiceTest.java:36)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:497)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
   	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
   	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
   	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
   	at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
   	at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33)
   	at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:230)
   	at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:58)
   Caused by: Error : 942, Position : 27, Sql = analyze table C##DEMO_DS_0.t_order_0 compute statistics, OriginalSql = analyze table C##DEMO_DS_0.t_order_0 compute statistics, Error Msg = ORA-00942: 表或视图不存在
   
   	at oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:513)
   	... 51 more
   ```
   When the table not exists, OracleDatabaseMetaData. GetIndexInfo will throw an exception, on the contrary the Mysql will not. 
   
   I added an isTableExist method in `IndexMetaDataLoader` to solve this issue as the `ColumnMetaDataLoader` do,  and it works.
   The source code is like the following,
   ```
       public static Collection<IndexMetaData> load(final Connection connection, final String table) throws SQLException {
           if (!isTableExist(connection, connection.getCatalog(), table)) {
               return Collections.emptyList();
           }
           Collection<IndexMetaData> result = new HashSet<>();
           try (ResultSet resultSet = connection.getMetaData().getIndexInfo(connection.getCatalog(), connection.getSchema(), table, false, false)) {
               while (resultSet.next()) {
                   String indexName = resultSet.getString(INDEX_NAME);
                   result.add(new IndexMetaData(indexName));
               }
           }
           return result;
       }
       
       private static boolean isTableExist(final Connection connection, final String catalog, final String table) throws SQLException {
           try (ResultSet resultSet = connection.getMetaData().getTables(catalog, null, table, null)) {
               return resultSet.next();
           }
       }
   ```
   Before commit a pr, I want to extract the isTableExist method from class IndexMetaDataLoader and ColumnMetaDataLoader. Could someone give me some advice about how to do the extract ? I will really appreciate any help on this, thanks.
   
   


----------------------------------------------------------------
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] TaoZhiMLND edited a comment on issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

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


   @lvye351  hi, I have some suggestions.
   1、check whether the oracle jvm installed?
   excute select * from dba_registry where comp_id = 'JAVAVM' with dba user. Is there a record?
   2、If not, install oracle jvm by database configuration assistant or executing $ORACLE_HOME/javavm/install/initjvm.sql.
   3、Then check whether can $ORACLE_HOME/javavm/install/initxa.sql execute successfully.
   


----------------------------------------------------------------
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] lvye351 commented on issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

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


    @cherrylzhao I used  latest   master brach  ,with oracle database 12c and ojbdc8.jar
   `  <groupId>com.oracle</groupId>
               <artifactId>ojdbc8</artifactId>
               <version>12.2.0.1.0</version>`
   
   sharding-databases-tables.yaml:
   
   dataSources:
     ds_0: !!com.zaxxer.hikari.HikariDataSource
       driverClassName: oracle.jdbc.OracleDriver
       jdbcUrl: jdbc:oracle:thin:@ip:1521:cc
       username: whl2
       password: whl2
     ds_1: !!com.zaxxer.hikari.HikariDataSource
       driverClassName: oracle.jdbc.OracleDriver
       jdbcUrl: jdbc:oracle:thin:@ip:1521:cc
       username: whl
       password: whl
   
   Caused by: oracle.jdbc.OracleDatabaseException: ORA-06550: 第 1 行, 第 14 列: 
   PLS-00201: identifier 'JAVA_XA.XA_START_NEW' must be declared
   ORA-06550: 第 1 行, 第 7 列: 
   PL/SQL: Statement ignored
   
   	at oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:498)
   	... 62 common frames omitted
   
   
   
   java.sql.SQLException: com.atomikos.icatch.jta.ExtendedSystemException: Unexpected error during enlist
   
   	at org.apache.shardingsphere.transaction.xa.XAShardingTransactionManager.getConnection(XAShardingTransactionManager.java:71)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.core.connection.ShardingConnection.createConnection(ShardingConnection.java:64)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.adapter.AbstractConnectionAdapter.createConnections(AbstractConnectionAdapter.java:126)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.adapter.AbstractConnectionAdapter.getConnections(AbstractConnectionAdapter.java:115)
   	at org.apache.shardingsphere.underlying.executor.sql.group.ExecuteGroupEngine.generateSQLExecuteGroups(ExecuteGroupEngine.java:105)
   	at org.apache.shardingsphere.underlying.executor.sql.group.ExecuteGroupEngine.generate(ExecuteGroupEngine.java:75)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.core.statement.ShardingPreparedStatement.executeUpdate(ShardingPreparedStatement.java:162)
   	at org.apache.shardingsphere.example.transaction.xa.raw.jdbc.XAOrderService.doInsert(XAOrderService.java:107)
   	at org.apache.shardingsphere.example.transaction.xa.raw.jdbc.XAOrderService.insert(XAOrderService.java:83)
   	at org.apache.shardingsphere.example.transaction.xa.raw.jdbc.XAOrderServiceTest.assertInsertSuccess(XAOrderServiceTest.java:47)
   
   
   is myql drvier version  or config wrong? 
   


----------------------------------------------------------------
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] cherrylzhao commented on issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

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


   I just google the error message, it seems that your oracle XA configuration is missed, see https://sourceforge.net/p/tyrex/discussion/180320/thread/eb505073/
   
   @TaoZhiMLND any thought here?


----------------------------------------------------------------
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] TaoZhiMLND commented on issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

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


   thanks a lot, I have commit the pr, let's have a look.


----------------------------------------------------------------
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] cherrylzhao commented on issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

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


   good job!
   I think we should move the `isTableExist` logic to TableMetaDataLoader, and the return type of TableMetaDataLoader#load should be `Optional<TableMetaData>`.
   
   ```java
   public final class TableMetaDataLoader {
       
       public static Optional<TableMetaData> load(final DataSource dataSource, final String table, final String databaseType) throws SQLException {
           try (Connection connection = dataSource.getConnection()) {
              if (!isTableExist(connection, connection.getCatalog(), table)) {
               return Optional.empty();
           }
               return Optional.of(new TableMetaData(ColumnMetaDataLoader.load(connection, table, databaseType), IndexMetaDataLoader.load(connection, table)));
           }
       }
       
       private static boolean isTableExist(final Connection connection, final String catalog, final String table) throws SQLException {
           try (ResultSet resultSet = connection.getMetaData().getTables(catalog, null, table, null)) {
               return resultSet.next();
           }
       }
   
   }
   ```
   
   committers for ShardingSphere, any thought?


----------------------------------------------------------------
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] lvye351 edited a comment on issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

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


    @cherrylzhao I used  latest   master brach  ,with oracle database 12c and ojbdc8.jar
   `  <groupId>com.oracle</groupId>
               <artifactId>ojdbc8</artifactId>
               <version>12.2.0.1.0</version>`
   
   sharding-databases-tables.yaml:
   
   dataSources:
     ds_0: !!com.zaxxer.hikari.HikariDataSource
       driverClassName: oracle.jdbc.OracleDriver
       jdbcUrl: jdbc:oracle:thin:@ip:1521:cc
       username: whl2
       password: whl2
     ds_1: !!com.zaxxer.hikari.HikariDataSource
       driverClassName: oracle.jdbc.OracleDriver
       jdbcUrl: jdbc:oracle:thin:@ip:1521:cc
       username: whl
       password: whl
   
   Caused by: oracle.jdbc.OracleDatabaseException: ORA-06550: 第 1 行, 第 14 列: 
   PLS-00201: identifier 'JAVA_XA.XA_START_NEW' must be declared
   ORA-06550: 第 1 行, 第 7 列: 
   PL/SQL: Statement ignored
   
   	at oracle.jdbc.driver.T4CTTIoer11.processError(T4CTTIoer11.java:498)
   	... 62 common frames omitted
   
   
   
   java.sql.SQLException: com.atomikos.icatch.jta.ExtendedSystemException: Unexpected error during enlist
   
   	at org.apache.shardingsphere.transaction.xa.XAShardingTransactionManager.getConnection(XAShardingTransactionManager.java:71)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.core.connection.ShardingConnection.createConnection(ShardingConnection.java:64)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.adapter.AbstractConnectionAdapter.createConnections(AbstractConnectionAdapter.java:126)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.adapter.AbstractConnectionAdapter.getConnections(AbstractConnectionAdapter.java:115)
   	at org.apache.shardingsphere.underlying.executor.sql.group.ExecuteGroupEngine.generateSQLExecuteGroups(ExecuteGroupEngine.java:105)
   	at org.apache.shardingsphere.underlying.executor.sql.group.ExecuteGroupEngine.generate(ExecuteGroupEngine.java:75)
   	at org.apache.shardingsphere.shardingjdbc.jdbc.core.statement.ShardingPreparedStatement.executeUpdate(ShardingPreparedStatement.java:162)
   	at org.apache.shardingsphere.example.transaction.xa.raw.jdbc.XAOrderService.doInsert(XAOrderService.java:107)
   	at org.apache.shardingsphere.example.transaction.xa.raw.jdbc.XAOrderService.insert(XAOrderService.java:83)
   	at org.apache.shardingsphere.example.transaction.xa.raw.jdbc.XAOrderServiceTest.assertInsertSuccess(XAOrderServiceTest.java:47)
   
   
   is my oracle drvier version  or config wrong? 
   


----------------------------------------------------------------
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] TaoZhiMLND commented on issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

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


   @lvye351  hi, I have some suggestions.
   1、check weather the oracle jvm installed?
   excute select * from dba_registry where comp_id = 'JAVAVM' with dba user. Is there a record?
   2、If not, install oracle jvm by database configuration assistant or executing $ORACLE_HOME/javavm/install/initjvm.sql.
   3、Then check whether can $ORACLE_HOME/javavm/install/initxa.sql execute successfully.
   


----------------------------------------------------------------
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] yu199195 commented on issue #5309: The junit test of transaction-2pc-xa-raw-jdbc-example module failed when a oracle database used

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


   good idea


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