You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by mi...@apache.org on 2010/07/11 18:12:45 UTC

svn commit: r963099 - in /db/derby/code/branches/10.4/java: engine/org/apache/derby/impl/sql/conn/ engine/org/apache/derby/jdbc/ testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/

Author: mikem
Date: Sun Jul 11 16:12:45 2010
New Revision: 963099

URL: http://svn.apache.org/viewvc?rev=963099&view=rev
Log:
DERBY-4731 XA two phase commit with active GLOBAL TEMPORARY TABLE causes An internal error identified by RawStore module

Backported fix #962738 from 10.6 codeline to 10.4 codeline.

For temp tables declared as following derby needs to do special work at the
time of commit to arrange for the temporary table to have no rows once the
commit completes.:
    DECLARE GLOBAL TEMPORARY TABLE SESSION.T1 ( XWSID INT) ON COMMIT DELETE ROWS NOT LOGGED ON ROLLBACK DELETE ROWS

Derby implements these temporary tables as unlogged real internal tables with
backing containers on disk. The ddl is all in memory so that they are only
seen by the current session. On commit the underlying container is dropped and
a new empty container is created.

This all works fine except in the XA case. In this case the transaction
has done real updates (temp table updates are unlogged and thus not seen
as update operations from the XA point of view), then the transaction executes
an XA prepare followed by an XA commit. No update transactions are allowed
between the prepare and the commit. The problem is that the pre-commit work
done for the temp tables was executing updates on the internal containers
(dropping and createing new ones), and raw store identified this as an XA
protocol violation. Since the work is only on internal non XA transaction
related updates it is ok to do these between the prepare and commit.

The fix arranges for this work to be done in a nested updatable user transaction
when in an XA transaction. It is ok to commit this work independently from
the parent user transaction because
for XA because it does the right thing in both possible cases:
    1) If the XA transaction commits successfully then the same work has been
       done. Because of where it is done in the code, the committing user can
       never get access to the global temp tables between the time the nested
       xact commits and the XA transaction commits.
    2) If the XA transaction fails to commit somehow, then I think one of two
       things will happen:
        a) the session will go away, and then it does not matter what happens
           to the session life objects.
        b) the transaction will rollback, and on commit work which deleted the
           rows is the same work that needs to get done on rollback.

The only locks the nested transaction gets is created new containers, so there
should be no problem with lock contention of the work with either the parent
transaction, or any other transactions. 


Modified:
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/jdbc/EmbedXAResource.java
    db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/XATest.java

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java?rev=963099&r1=963098&r2=963099&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java Sun Jul 11 16:12:45 2010
@@ -574,25 +574,112 @@ public class GenericLanguageConnectionCo
 		}
 	}
 
-	/**
-	 * do the necessary work at commit time for temporary tables
-	 * 1)If a temporary table was marked as dropped in this transaction, then remove it from the list of temp tables for this connection
-	 * 2)If a temporary table was not dropped in this transaction, then mark it's declared savepoint level and modified savepoint level as -1
-	 */
-	private void tempTablesAndCommit() {
-		for (int i = allDeclaredGlobalTempTables.size()-1; i >= 0; i--) {
-			TempTableInfo tempTableInfo = (TempTableInfo)allDeclaredGlobalTempTables.get(i);
-			if (tempTableInfo.getDroppedInSavepointLevel() != -1)
-			{
-				//this means table was dropped in this unit of work and hence should be removed from valid list of temp tables
-				allDeclaredGlobalTempTables.remove(i);
-			} else //this table was not dropped in this unit of work, hence set its declaredInSavepointLevel as -1 and also mark it as not modified 
-			{
-				tempTableInfo.setDeclaredInSavepointLevel(-1);
-				tempTableInfo.setModifiedInSavepointLevel(-1);
-			}
-		}
-	}
+    /**
+     * Do the necessary work at commit time for temporary tables
+     * <p>
+     * 1)If a temporary table was marked as dropped in this transaction, then 
+     *   remove it from the list of temp tables for this connection
+     * 2)If a temporary table was not dropped in this transaction, then mark 
+     *   it's declared savepoint level and modified savepoint level as -1
+     * 3)After savepoint fix up, then handle all ON COMMIT DELETE ROWS with
+     *   no open held cursor temp tables.
+     * <p>
+     *
+     * @param in_xa_transaction if true, then transaction is an XA transaction,
+     *                          and special nested transaction may be necessary
+     *                          to cleanup internal containers supporting the
+     *                          temp tables at commit time.
+     *
+     * @exception  StandardException  Standard exception policy.
+     **/
+    private void tempTablesAndCommit(boolean in_xa_transaction) 
+        throws StandardException
+    {
+        // loop through all declared global temporary tables and determine
+        // what to do at commit time based on if they were dropped during
+        // the current savepoint level.
+        for (int i = allDeclaredGlobalTempTables.size()-1; i >= 0; i--) 
+        {
+            TempTableInfo tempTableInfo = 
+                (TempTableInfo)allDeclaredGlobalTempTables.get(i);
+
+            if (tempTableInfo.getDroppedInSavepointLevel() != -1)
+            {
+                // this means table was dropped in this unit of work and hence 
+                // should be removed from valid list of temp tables
+
+                allDeclaredGlobalTempTables.remove(i);
+            } 
+            else 
+            {
+                //this table was not dropped in this unit of work, hence set 
+                //its declaredInSavepointLevel as -1 and also mark it as not 
+                //modified 
+
+                tempTableInfo.setDeclaredInSavepointLevel(-1);
+                tempTableInfo.setModifiedInSavepointLevel(-1);
+            }
+        }
+
+        // at commit time, for all the temp tables declared with 
+        // ON COMMIT DELETE ROWS, make sure there are no held cursor open
+        // on them.
+        // If there are no held cursors open on ON COMMIT DELETE ROWS, 
+        // drop those temp tables and redeclare them to get rid of all the 
+        // data in them
+
+        // in XA use nested user updatable transaction.  Delay creating
+        // the transaction until loop below finds one it needs to 
+        // process.
+        TransactionController xa_tran       = null; 
+        TransactionController tran_for_drop = 
+            (in_xa_transaction ? null : getTransactionExecute());
+
+        try
+        {
+            for (int i=0; i<allDeclaredGlobalTempTables.size(); i++)
+            {
+                TableDescriptor td = 
+                    ((TempTableInfo) (allDeclaredGlobalTempTables.
+                                          get(i))).getTableDescriptor();
+                if (td.isOnCommitDeleteRows() == false) 
+                {
+                    // do nothing for temp table with ON COMMIT PRESERVE ROWS
+                    continue;
+                }
+                else if (checkIfAnyActivationHasHoldCursor(td.getName()) == 
+                            false)
+                {
+                    // temp tables with ON COMMIT DELETE ROWS and 
+                    // no open held cursors
+                    getDataDictionary().getDependencyManager().invalidateFor(
+                        td, DependencyManager.DROP_TABLE, this);
+
+                    // handle delayed creation of nested xact for XA.
+                    if (in_xa_transaction)
+                    {
+                        if (xa_tran == null)
+                        {
+                            xa_tran = 
+                                getTransactionExecute().
+                                    startNestedUserTransaction(false);
+                            tran_for_drop = xa_tran;
+                        }
+                    }
+
+                    cleanupTempTableOnCommitOrRollback(tran_for_drop, td, true);
+                }
+            }
+        }
+        finally
+        {
+            // if we created a nested user transaction for XA get rid of it.
+            if (xa_tran != null)
+            {
+                xa_tran.destroy();
+            }
+        }
+    }
 
 	/**
 		Reset the connection before it is returned (indirectly) by
@@ -701,7 +788,8 @@ public class GenericLanguageConnectionCo
 			{
 				//restore the old definition of temp table because drop is being rolledback
 				TableDescriptor td = tempTableInfo.getTableDescriptor();
-				td = cleanupTempTableOnCommitOrRollback(td, false);
+				td = cleanupTempTableOnCommitOrRollback(
+                        getTransactionExecute(), td, false);
 				//In order to store the old conglomerate information for the temp table, we need to replace the
 				//existing table descriptor with the old table descriptor which has the old conglomerate information
 				tempTableInfo.setTableDescriptor(td);
@@ -715,7 +803,8 @@ public class GenericLanguageConnectionCo
 				tempTableInfo.setModifiedInSavepointLevel(-1);
 				TableDescriptor td = tempTableInfo.getTableDescriptor();
 				getDataDictionary().getDependencyManager().invalidateFor(td, DependencyManager.DROP_TABLE, this);
-				cleanupTempTableOnCommitOrRollback(td, true);
+				cleanupTempTableOnCommitOrRollback(
+                        getTransactionExecute(), td, true);
 			} // there is no else here because there is no special processing required for temp tables declares in earlier work of unit/transaction and not modified
 		}
     
@@ -1174,31 +1263,11 @@ public class GenericLanguageConnectionCo
 
 		endTransactionActivationHandling(false);
 
-		//do the clean up work required for temporary tables at the commit time. This cleanup work
-		//can possibly remove entries from allDeclaredGlobalTempTables and that's why we need to check
-		//again later to see if we there are still any entries in allDeclaredGlobalTempTables
-		if (allDeclaredGlobalTempTables != null)
-		{
-			tempTablesAndCommit();
-			//at commit time, for all the temp tables declared with ON COMMIT DELETE ROWS, make sure there are no held cursor open on them.
-			//If there are no held cursors open on ON COMMIT DELETE ROWS, drop those temp tables and redeclare them to get rid of all the data in them
-			if (allDeclaredGlobalTempTables != null) {
-				for (int i=0; i<allDeclaredGlobalTempTables.size(); i++)
-				{
-					TableDescriptor td = ((TempTableInfo)(allDeclaredGlobalTempTables.get(i))).getTableDescriptor();
-					if (td.isOnCommitDeleteRows() == false) //do nothing for temp table with ON COMMIT PRESERVE ROWS
-					{
-						continue;
-					}
-					if (checkIfAnyActivationHasHoldCursor(td.getName()) == false)//temp tables with ON COMMIT DELETE ROWS and no open held cursors
-					{
-						getDataDictionary().getDependencyManager().invalidateFor(td, DependencyManager.DROP_TABLE, this);
-						cleanupTempTableOnCommitOrRollback(td, true);
-					}
-				}
-			}
-		}
-
+        // Do clean up work required for temporary tables at commit time.  
+        if (allDeclaredGlobalTempTables != null)
+        {
+            tempTablesAndCommit(commitflag != NON_XA);
+        }
 
 		currentSavepointLevel = 0; //reset the current savepoint level for the connection to 0 at the end of commit work for temp tables
 
@@ -1269,14 +1338,15 @@ public class GenericLanguageConnectionCo
      * temp table (because the drop on it is being rolled back).
 	 */
 	private TableDescriptor cleanupTempTableOnCommitOrRollback(
-    TableDescriptor td, 
-    boolean         dropAndRedeclare)
+    TransactionController   tc,
+    TableDescriptor         td, 
+    boolean                 dropAndRedeclare)
 		 throws StandardException
 	{
 		//create new conglomerate with same properties as the old conglomerate 
         //and same row template as the old conglomerate
 		long conglomId = 
-            tran.createConglomerate(
+            tc.createConglomerate(
                 "heap", // we're requesting a heap conglomerate
                 td.getEmptyExecRow().getRowArray(), // row template
                 null, //column sort order - not required for heap
@@ -1301,7 +1371,7 @@ public class GenericLanguageConnectionCo
 
 		if(dropAndRedeclare)
 		{
-			tran.dropConglomerate(cid); //remove the old conglomerate from the system
+			tc.dropConglomerate(cid); //remove the old conglomerate from the system
 			replaceDeclaredGlobalTempTable(td.getName(), td);
 		}
 

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/jdbc/EmbedXAResource.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/jdbc/EmbedXAResource.java?rev=963099&r1=963098&r2=963099&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/jdbc/EmbedXAResource.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/jdbc/EmbedXAResource.java Sun Jul 11 16:12:45 2010
@@ -290,10 +290,16 @@ class EmbedXAResource implements XAResou
                 } else {
                     
                     returnConnectionToResource(tranState, xid_im);
+
 					if (SanityManager.DEBUG) {
-						if (con.realConnection != null)
-							SanityManager.ASSERT(con.realConnection.transactionIsIdle(),
-									"real connection should have been idle at this point"); 			
+						if (con.realConnection != null) {
+							SanityManager.ASSERT(
+                                con.realConnection.transactionIsIdle(),
+                                "real connection should have been idle." +
+                                "tranState = " + tranState +
+                                "ret = " + ret +
+                                "con.realConnection = " + con.realConnection);
+                        }
 					}
                     return XAResource.XA_RDONLY;
                 }

Modified: db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/XATest.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/XATest.java?rev=963099&r1=963098&r2=963099&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/XATest.java (original)
+++ db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/XATest.java Sun Jul 11 16:12:45 2010
@@ -1079,6 +1079,106 @@ public class XATest extends BaseJDBCTest
     }
 
     /**
+     * DERBY-4731
+     * Test using a GLOBAL TEMPORARY TABLE  table in an
+     * XA transaction and leaving it active during two phase commit.
+     * Before the fix this test would throw the following at commit
+     * time:
+     * ERROR 40XT0: An internal error was identified by RawStore module. 
+     *
+     *
+     * @throws XAException 
+     * @throws SQLException 
+     * 
+     */
+    public void testXATempTableD4731_RawStore() 
+        throws SQLException, XAException {
+        doXATempTableD4731Work(true, XATestUtil.getXid(997, 9, 49));
+    }
+    
+
+    /**
+     * DERBY-XXXX Temp tables with XA transactions
+     * an Assert will occur on prepare if only
+     * temp table work is done in the xact.
+     *
+     * @throws XAException 
+     * @throws SQLException 
+     * 
+     */
+    public void xtestXATempTableDXXXX_Assert() 
+        throws SQLException, XAException {
+
+          doXATempTableD4731Work(false, XATestUtil.getXid(998, 10, 50));
+    }
+ 
+    
+    /**
+     * The two cases for DERBY-4371 do essentially the same thing. Except doing
+     * logged work causes the RawStore error and doing only temp table 
+     * operations causes the assert.
+     *  
+     * @param doLoggedWorkInXact
+     * @throws SQLException
+     * @throws XAException
+     */
+    private void doXATempTableD4731Work(
+    boolean doLoggedWorkInXact,
+    Xid     xid)
+        throws SQLException, XAException{
+
+        XADataSource xads = J2EEDataSource.getXADataSource();
+        XAConnection xaconn = xads.getXAConnection();
+        XAResource xar = xaconn.getXAResource();
+
+        xar.start(xid, XAResource.TMNOFLAGS);
+        Connection conn = xaconn.getConnection();
+        Statement s = conn.createStatement(); 
+        if (doLoggedWorkInXact){
+            // need to do some real work in our transaction
+            // so make a table
+            makeARealTable(s);
+        }
+        
+        // make the temp table
+        s.executeUpdate("DECLARE GLOBAL TEMPORARY TABLE SESSION.T1 ( XWSID INT, XCTID INT, XIID CHAR(26), XVID SMALLINT, XLID CHAR(8) FOR BIT DATA) ON COMMIT DELETE ROWS NOT LOGGED ON ROLLBACK DELETE ROWS");
+
+        // insert a row
+        PreparedStatement ps = 
+            conn.prepareStatement("INSERT INTO SESSION.T1 VALUES (?,?,?,?,?)");
+        ps.setInt(1,1);
+        ps.setInt(2,1);
+        ps.setString(3,"hello");
+        ps.setShort(4, (short) 1);
+        ps.setBytes(5, new byte[] {0x0,0x1});
+        ps.executeUpdate();
+        ResultSet rs = s.executeQuery("SELECT count(*) FROM SESSION.t1");
+        JDBC.assertFullResultSet(rs, new String[][] {{"1"}});
+        // You could work around the issue by dropping the TEMP table
+        //s.executeUpdate("DROP TABLE SESSION.T1");
+        xar.end(xid, XAResource.TMSUCCESS);
+
+        assertEquals(
+            (doLoggedWorkInXact ? XAResource.XA_OK : XAResource.XA_RDONLY),
+            xar.prepare(xid));
+
+        xar.commit(xid,false); 
+        s.close();
+        conn.close();
+        xaconn.close();
+    }
+
+    private void makeARealTable(Statement s) throws SQLException {
+        try {
+            s.executeUpdate("DROP TABLE REALTABLE1");
+        } catch (SQLException se) {
+            {
+            s.executeUpdate("CREATE TABLE REALTABLE1 (i int)");
+            }
+        }
+    }
+    
+    /**
      * Check the held state of a ResultSet by fetching one row, executing a
      * commit and then fetching the next. Checks the held state matches the
      * behaviour.