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 2006/01/19 17:32:17 UTC

svn commit: r370530 - in /db/derby/code/trunk/java: engine/org/apache/derby/catalog/ engine/org/apache/derby/iapi/reference/ engine/org/apache/derby/iapi/store/access/ engine/org/apache/derby/iapi/store/raw/ engine/org/apache/derby/iapi/store/raw/xact/...

Author: mikem
Date: Thu Jan 19 08:32:08 2006
New Revision: 370530

URL: http://svn.apache.org/viewcvs?rev=370530&view=rev
Log:
DERBY-239, most recent patch committed on behalf of Suresh Thalamati

This patch addresses the issues raised by ·¹ystein in his review of previous
online backup patches 3-6. 

- changed the backup procedures names with ONLINE to NOWAIT 
- removed the transaction Idle restriction to run backup procedures.
- removed implicit commit/rollbacks.
- Added a new lesser impact restriction, which only disallows backup call only if
  there are unlogged operations executed in the same transaction before the backup. 
  - Removed casting to  RawTransaction. 
  - fixed Names and Comments. 
  - Enhanced the tests with addional test cases suggested by  ·¹ystein.


Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/SQLState.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/RawTransaction.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RFResource.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest1.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest3.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/onlineBackupTest2.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest3.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/onlineBackupTest2.sql

Modified: db/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/catalog/SystemProcedures.java Thu Jan 19 08:32:08 2006
@@ -684,13 +684,16 @@
     }
 
     /**
-     * Backup the database to a backup directory. By default this 
-     * procedure will wait for the backup blocking unlogged operations to
-     * complete before starting the backup.  
+     * Backup the database to a backup directory. 
+     *
+     * This procedure will throw error, if there are any unlogged 
+     * operation executed in the same transaction backup is started.
+     * If there any unlogged operations in progess in other transaction, it
+     * will wait until those transactions are completed before starting the backup.
      *
      * Examples of unlogged operations include: create index and bulk insert.
      * Note that once the backup begins these operations will not block, 
-     * instead they will automatically procede and be logged.
+     * instead they are automatically converted into logged operations.
      * 
      * @param backupDir the name of the directory where the backup should be
      *                  stored. This directory will be created if it 
@@ -700,83 +703,45 @@
     public static void SYSCS_BACKUP_DATABASE(String  backupDir)
 		throws SQLException
     {
-		backupDatabase(backupDir , true);
+        Factory.getDatabaseOfConnection().backup(backupDir, true);
     }
 
     /**
      * Backup the database to a backup directory.
      *
-     * Backup the database to a backup directory. Use the "wait" parameter
-     * to determine if the operation should block waiting for the backup 
-     * blocking unlogged operations to complete before starting the backup.  
-     *
-     * Examples of unlogged operations include: create index and bulk insert.
-     * Note that once the backup begins these operations will not block, 
-     * instead they will automatically procede and be logged.
+     * This procedure will throw error, if there are any uncommitted unlogged 
+     * operation before stating the backup. It will not wait for the unlogged
+     * operations to complete.
      * 
-     * @param backupDir the name of the directory where the backup should be
-     *                  stored. This directory will be created if it 
-     *                  does not exist.
-     * @param wait if <tt>non-zero</tt>, waits for  all the backup blocking 
-     *             operation in progress to finish.
-     * @exception StandardException thrown on error
-     */
-    public static void SYSCS_ONLINE_BACKUP_DATABASE(
-    String  backupDir, 
-    int     wait)
-        throws SQLException
-    {
-        backupDatabase(backupDir, (wait != 0));
-    }
-
-
-    /**
-     * Backup the database to a backup directory.
-     *
-     * Backup the database to a backup directory. Use the "wait" parameter
-     * to determine if the operation should block waiting for the backup 
-     * blocking unlogged operations to complete before starting the backup.  
-     *
      * Examples of unlogged operations include: create index and bulk insert.
      * Note that once the backup begins these operations will not block, 
-     * instead they will automatically procede and be logged.
+     * instead they are automatically converted into logged operations.
      * 
      * @param backupDir the name of the directory where the backup should be
      *                  stored. This directory will be created if it 
      *                  does not exist.
-     * @param wait if <tt>true</tt>, waits for  all the backup blocking 
-     *             operation in progress to finish.
      * @exception StandardException thrown on error
      */
-    private static void backupDatabase(
-    String  backupDir, 
-    boolean wait)
+    public static void SYSCS_BACKUP_DATABASE_NOWAIT(String  backupDir)
         throws SQLException
     {
-        checkBackupTransactionIsIdle();
-        Connection conn = getDefaultConn();
-        try {
-            Factory.getDatabaseOfConnection().backup(backupDir, wait);
-        }catch(SQLException se)
-        {
-            // issue a rollback on any errors
-            conn.rollback();
-            throw  se;
-        }
-        // finished successfully, commit it.
-        conn.commit();	
+        Factory.getDatabaseOfConnection().backup(backupDir, false);
     }
 
+
     /**
      * Backup the database to a backup directory and enable the log archive
      * mode that will keep the archived log files required for roll-forward
-     * from this version of the backup. By default this procedure will wait 
-     * for the backup blocking unlogged operations to complete before starting 
-     * the backup.  
+     * from this version of the backup.
+     *
+     * This procedure will throw error if there are any unlogged 
+     * operation executed in the same transaction backup is started.
+     * If there any unlogged operations in progess in other transaction, it
+     * will wait until those transactions are completed before starting the backup.
      *
      * Examples of unlogged operations include: create index and bulk insert.
      * Note that once the backup begins these operations will not block, 
-     * instead they will automatically procede and be logged.
+     * instead they are automatically converted into logged operations.
      *
      * @param backupDir the name of the directory where the backup should be
      *                  stored. This directory will be created if not it 
@@ -791,10 +756,11 @@
     int     deleteOnlineArchivedLogFiles)
 		throws SQLException
     {
-        backupDatabaseAndEnableLogArchiveMode(
-            backupDir, 
-            (deleteOnlineArchivedLogFiles != 0),
-            true);
+
+        Factory.getDatabaseOfConnection().backupAndEnableLogArchiveMode(
+                backupDir, 
+                (deleteOnlineArchivedLogFiles != 0),
+                true);
 	}
 
     /**
@@ -802,69 +768,36 @@
 	 * mode that will keep the archived log files required for roll-forward
 	 * from this version backup.
      *
+     * This procedure will throw error, if there are any uncommitted unlogged 
+     * operation before stating the backup. It will not wait for the unlogged
+     * operations to complete.
+     * 
+
      * Examples of unlogged operations include: create index and bulk insert.
      * Note that once the backup begins these operations will not block, 
-     * instead they will automatically procede and be logged.
+     * instead they are automatically converted into logged operations.
      *
      * @param backupDir the name of the directory where the backup should be
      *                  stored. This directory will be created if not it 
      *                  does not exist.   
      *
      * @param deleteOnlineArchivedLogFiles  If <tt>non-zero</tt> deletes online 
-     *                  archived log files that exist before this backup, delete     *                  will occur  only after the backup is  complete.
+     *                  archived log files that exist before this backup, delete     
+     *                  will occur  only after the backup is  complete.
      *
-     * @param wait      if <tt>non-zero</tt>, waits for  all the backup blocking     *                  operations in progress to finish.
      * @exception StandardException thrown on error.
      */
-    public static void SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
+    public static void SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT(
     String  backupDir,
-    int     deleteOnlineArchivedLogFiles,
-	int     wait)
+    int     deleteOnlineArchivedLogFiles)
 		throws SQLException
     {
-        backupDatabaseAndEnableLogArchiveMode(
-            backupDir,
-            (deleteOnlineArchivedLogFiles != 0),
-            (wait != 0));
-	}
-
 
-    /**
-     * Backup the database to a backup directory and enable the log archive
-	 * mode that will keep the archived log files required for roll-forward
-	 * from this version of the backup.
-     *
-     * @param backupDir the name of the directory where the backup should be
-     *                  stored. This directory will be created if not it 
-     *                  does not exist.   
-     * @param deleteOnlineArchivedLogFiles  If <tt>true</tt> deletes online 
-     *                  archived log files that exist before this backup, delete     *                  will occur  only after the backup is  complete.
-	 * @param wait      if <tt>true</tt>, waits for  all the backup blocking 
-	 *                  operations in progress to finish.
-     * @exception StandardException thrown on error.
-     */
-    private static void backupDatabaseAndEnableLogArchiveMode(
-    String  backupDir,
-    boolean deleteOnlineArchivedLogFiles,
-	boolean wait)
-		throws SQLException
-    {
-		checkBackupTransactionIsIdle();
-		Connection conn = getDefaultConn();
-		try {
-            Factory.getDatabaseOfConnection().backupAndEnableLogArchiveMode(
-                backupDir, 
-                deleteOnlineArchivedLogFiles,
-                wait);
-		}catch(SQLException se)
-		{
-            // issue a rollback on any errors
-            conn.rollback();
-            throw  se;
-        }
-        // finished successfully, commit it.
-        conn.commit();
-    }
+        Factory.getDatabaseOfConnection().backupAndEnableLogArchiveMode(
+                backupDir,
+                (deleteOnlineArchivedLogFiles != 0),
+                false);
+	}
 
 
     /**
@@ -881,35 +814,9 @@
     int     deleteOnlineArchivedLogFiles)
 		throws SQLException
     {
-		checkBackupTransactionIsIdle();
-		Connection conn = getDefaultConn();
-		try { 
-			Factory.getDatabaseOfConnection().disableLogArchiveMode(
+        Factory.getDatabaseOfConnection().disableLogArchiveMode(
                 (deleteOnlineArchivedLogFiles != 0));
-		}catch(SQLException se)
-		{
-			// issue a rollback on any errors
-			conn.rollback();
-			throw  se;
-		}
-		// finished successfully, commit it.
-		conn.commit();
     }
-
-
-    /**
-     * Check if the transnaction is idle ? , Backup related operation are 
-     * allowed only in a new transaction.
-     */
-    private static void checkBackupTransactionIsIdle() throws SQLException 
-    {
-        if (!(ConnectionUtil.getCurrentLCC().getTransactionExecute().isIdle())) 
-        {
-            throw PublicAPI.wrapStandardException(
-                    StandardException.newException(
-                    SQLState.BACKUP_OPERATIONS_NOT_ALLOWED_IN_ACTIVE_XACT));								  
-		}
-	}
 
 
     public static void SYSCS_SET_RUNTIMESTATISTICS(

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/SQLState.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/SQLState.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/SQLState.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/SQLState.java Thu Jan 19 08:32:08 2006
@@ -357,7 +357,7 @@
     String RAWSTORE_CANNOT_CHANGE_LOGDEVICE                     = "XSRS8.S";
     String RAWSTORE_RECORD_VANISHED                             = "XSRS9.S";
     String BACKUP_BLOCKING_OPERATIONS_IN_PROGRESS               = "XSRSA.S";
-    String BACKUP_OPERATIONS_NOT_ALLOWED_IN_ACTIVE_XACT         = "XSRSB.S";
+    String BACKUP_OPERATIONS_NOT_ALLOWED                        = "XSRSB.S";
     
 
 	/*

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java Thu Jan 19 08:32:08 2006
@@ -215,7 +215,7 @@
 
 	/**
 	  * Freeze the database temporarily so a backup can be taken.
-	  * <P>Please see cloudscape on line documentation on backup and restore.
+	  * <P>Please see Derby on line documentation on backup and restore.
 	  *
 	  * @exception StandardException Thrown on error
 	  */
@@ -223,7 +223,7 @@
 
 	/**
 	  * Unfreeze the database after a backup has been taken.
-	  * <P>Please see cloudscape on line documentation on backup and restore.
+	  * <P>Please see Derby on line documentation on backup and restore.
 	  *
 	  * @exception StandardException Thrown on error
 	  */
@@ -231,7 +231,7 @@
 
 	/**
      * Backup the database to backupDir.  
-     * <P>Please see cloudscape on line documentation on backup and restore.
+     * <P>Please see Derby on line documentation on backup and restore.
      *
      * @param backupDir the name of the directory where the backup should be
      *		             stored.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java Thu Jan 19 08:32:08 2006
@@ -27,7 +27,7 @@
 import org.apache.derby.iapi.services.property.PersistentSet;
 
 import org.apache.derby.iapi.store.access.TransactionInfo;
-
+import org.apache.derby.iapi.store.raw.xact.TransactionFactory;
 import org.apache.derby.iapi.store.raw.log.LogInstant;
 import org.apache.derby.iapi.error.StandardException;
 
@@ -878,4 +878,10 @@
      **/
     long getMaxContainerId()
 		throws StandardException;
+
+
+    /**
+		Get the Transaction Factory to use with this store.
+	*/
+	public TransactionFactory getXactFactory();
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/RawTransaction.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/RawTransaction.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/RawTransaction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/RawTransaction.java Thu Jan 19 08:32:08 2006
@@ -330,19 +330,24 @@
 
 	
     /*
-     * Try setting the transaction to be in backup blocking state.
+     * Make the transaction block the online backup.
      *
      * @param wait if <tt>true</tt>, waits until the transaction
-     *             can be set into backup blocking state.
-     * @return     <tt>true</tt> if the transaction can be set to a 
-     *             blocking state. 
-     *             <tt>false</tt> otherwise.
+     *             can block the backup.
+     * @return     <tt>true</tt> if the transaction  blocked the  
+     *             backup.  <tt>false</tt> otherwise.
      * @exception StandardException if interrupted while waiting 
-     *            for backup to complete to set the transaction into
-     *            backup blocking state.
+     *            for the backup in progress to complete.
      */
-    public abstract boolean setBackupBlockingState(boolean wait)
+    public abstract boolean blockBackup(boolean wait)
         throws StandardException;
+
+    /**
+     * Check if the transaction is blocking the backup ?
+     * @return <tt> true </tt> if this transaction is 
+     *         blocking the backup, otherwise <tt> false </tt>
+     */
+    public abstract boolean isBlockingBackup();
 
 }
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java Thu Jan 19 08:32:08 2006
@@ -298,22 +298,22 @@
 
 	/**
 	 * Checks if there any backup blocking operations are in progress and 
-	 * stops from new ones from starting until the backup is finished. 
+	 * prevents new ones from starting until the backup is finished. 
 	 *
 	 * @param wait if <tt>true</tt>, waits for the current backup blocking 
-	 *             operation in progress to finish.
+	 *             operations in progress to finish.
 	 * @return     <tt>true</tt> if there are no backup blocking 
 	 *             operations in progress.
 	 *             <tt>false</tt> otherwise.
 	 * @exception StandardException if interrupted or a runtime exception occurs.
 	 */
-	public boolean stopBackupBlockingOperations(boolean wait) 
+	public boolean blockBackupBlockingOperations(boolean wait) 
 		throws StandardException ;
 
 	/**
 	 * Backup completed. Allow backup blocking operations. 
 	 */
-	public void backupFinished();
+	public void  unblockBackupBlockingOperations();
 
 													   
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java Thu Jan 19 08:32:08 2006
@@ -349,10 +349,9 @@
 
 
 
-    public void backupAndEnableLogArchiveMode(
-    String  backupDir, 
-    boolean deleteOnlineArchivedLogFiles,
-    boolean wait)
+    public void backupAndEnableLogArchiveMode(String  backupDir, 
+                                              boolean deleteOnlineArchivedLogFiles,
+                                              boolean wait)
         throws SQLException
 	{
 		try {

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java Thu Jan 19 08:32:08 2006
@@ -9494,25 +9494,22 @@
 		throws StandardException
     {
 
-        // void SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE(
-        //     IN BACKUPDIR        VARCHAR(Limits.DB2_VARCHAR_MAXWIDTH),
-        //     IN WAIT             SMALLINT
+        // void SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT(
+        //     IN BACKUPDIR        VARCHAR(Limits.DB2_VARCHAR_MAXWIDTH)
         //     )
         
         {
             // procedure argument names
-            String[] arg_names = {"BACKUPDIR", "WAIT"};
+            String[] arg_names = {"BACKUPDIR"};
 
             // procedure argument types
             TypeDescriptor[] arg_types = {
                 DataTypeDescriptor.getBuiltInDataTypeDescriptor(
-                    Types.VARCHAR, Limits.DB2_VARCHAR_MAXWIDTH),
-                DataTypeDescriptor.getBuiltInDataTypeDescriptor(
-                    Types.SMALLINT)
+                    Types.VARCHAR, Limits.DB2_VARCHAR_MAXWIDTH)
             };
 
             createSystemProcedureOrFunction(
-                "SYSCS_ONLINE_BACKUP_DATABASE",
+                "SYSCS_BACKUP_DATABASE_NOWAIT",
                 sysUtilUUID,
                 arg_names,
                 arg_types,
@@ -9524,28 +9521,25 @@
         }
 
         // void 
-        // SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
+        // SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT(
         //   IN BACKUPDIR                 VARCHAR(Limits.DB2_VARCHAR_MAXWIDTH),
-        //   IN DELETE_ARCHIVED_LOG_FILES SMALLINT,
-        //   IN WAIT                      SMALLINT
+        //   IN DELETE_ARCHIVED_LOG_FILES SMALLINT
         //   )
         {
             // procedure argument names
             String[] arg_names = 
-                {"BACKUPDIR", "DELETE_ARCHIVED_LOG_FILES", "WAIT"};
+                {"BACKUPDIR", "DELETE_ARCHIVED_LOG_FILES"};
 
             // procedure argument types
             TypeDescriptor[] arg_types = {
                 DataTypeDescriptor.getBuiltInDataTypeDescriptor(
                     Types.VARCHAR, Limits.DB2_VARCHAR_MAXWIDTH),
                 DataTypeDescriptor.getBuiltInDataTypeDescriptor(
-                    Types.SMALLINT),
-                DataTypeDescriptor.getBuiltInDataTypeDescriptor(
                     Types.SMALLINT)
             };
 
             createSystemProcedureOrFunction(
-                "SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE",
+                "SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT",
                 sysUtilUUID,
                 arg_names,
                 arg_types,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java Thu Jan 19 08:32:08 2006
@@ -42,6 +42,7 @@
 import org.apache.derby.iapi.store.raw.ScanHandle;
 import org.apache.derby.iapi.store.raw.RawStoreFactory;
 import org.apache.derby.iapi.store.raw.Transaction;
+import org.apache.derby.iapi.store.raw.xact.RawTransaction;
 import org.apache.derby.iapi.store.raw.xact.TransactionFactory;
 import org.apache.derby.iapi.store.raw.data.DataFactory;
 import org.apache.derby.iapi.store.raw.log.LogFactory;
@@ -377,6 +378,14 @@
 		return xactFactory.getLockFactory();
 	}
 
+    
+    /**
+		Get the Transaction Factory to use with this store.
+	*/
+	public TransactionFactory getXactFactory() {
+        return xactFactory;  
+    }
+
 	/*
 	 * Return the module providing XAresource interface to the transaction
      * table.
@@ -492,15 +501,28 @@
 
 		// find the user transaction, it is necessary for online backup 
 		// to open the container through page cache
-		Transaction t = 
-            findUserTransaction(
+		RawTransaction t = 
+            xactFactory.findUserTransaction(this,
                 ContextService.getFactory().getCurrentContextManager(), 
                 AccessFactoryGlobals.USER_TRANS_NAME);
 
 		try {
-            // check if there any backup blocking operations are in progress
+
+            // check if  any backup blocking operations are in progress
+            // in the same transaction backup is being executed? Backup is 
+            // not allowed if the transaction has uncommitted
+            // unlogged operations that are blocking the backup.
+            
+            if (t.isBlockingBackup())
+            {
+                throw StandardException.newException(
+                      SQLState.BACKUP_OPERATIONS_NOT_ALLOWED);  
+            }
+
+
+            // check if any backup blocking operations are in progress
             // and stop new ones from starting until the backup is completed.
-            if (!xactFactory.stopBackupBlockingOperations(wait))
+            if (!xactFactory.blockBackupBlockingOperations(wait))
             {
                 throw StandardException.newException(
                       SQLState.BACKUP_BLOCKING_OPERATIONS_IN_PROGRESS);  
@@ -511,7 +533,7 @@
         }finally {
             // let the xactfatory know that backup is done, so that
             // it can allow backup blocking operations. 
-            xactFactory.backupFinished();
+            xactFactory.unblockBackupBlockingOperations();
         }
 	}
 
@@ -812,7 +834,6 @@
 		throws StandardException
 	{
         boolean enabledLogArchive = false;
-        boolean error = true;
         try {
             // Enable the log archive mode, if it is not already enabled.
             if(!logFactory.logArchived()) {
@@ -829,14 +850,12 @@
             {
                 logFactory.deleteOnlineArchivedLogFiles();
             }
-            error = false;
-        } finally {
+        }catch (Throwable error) {
             // On any errors , disable the log archive, if it 
             // is enabled on this call. 
-            if(error) {
-                if (enabledLogArchive)
+            if (enabledLogArchive)
                 logFactory.disableLogArchiveMode();
-            }
+            throw StandardException.plainWrapException(error);
         }
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java Thu Jan 19 08:32:08 2006
@@ -662,10 +662,10 @@
 						((mode & ContainerHandle.MODE_CREATE_UNLOGGED) == 
 						 ContainerHandle.MODE_CREATE_UNLOGGED))									   
 					{
-						if (!t.setBackupBlockingState(false)) {
+						if (!t.blockBackup(false)) {
 							// when a backup is in progress transaction can not
-                            // be set to backup blocking state, so convert 
-                            // unlogged opens to logged mode.
+                            // block the backup, so convert  unlogged opens 
+                            // to logged mode.
 							mode &= ~(ContainerHandle.MODE_UNLOGGED |
 									  ContainerHandle.MODE_CREATE_UNLOGGED);
 						}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RFResource.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RFResource.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RFResource.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RFResource.java Thu Jan 19 08:32:08 2006
@@ -79,15 +79,19 @@
             ContextManager cm = 
                 ContextService.getFactory().getCurrentContextManager();
 
-            Transaction tran = 
-                factory.getRawStoreFactory().findUserTransaction(
-                        cm, AccessFactoryGlobals.USER_TRANS_NAME);
+            RawTransaction tran = 
+                factory.getRawStoreFactory().getXactFactory().findUserTransaction(
+                        factory.getRawStoreFactory(), 
+                        cm, 
+                        AccessFactoryGlobals.USER_TRANS_NAME);
             
-            // Prevent backup operation when a jar file is being added
-            // by setting the transaction into a backup blocking state.
-            // If backup is already in progress this call will wait 
-            // for the backup to finish .
-            ((RawTransaction)tran).setBackupBlockingState(true);
+            // Block the backup, If backup is already in progress wait 
+            // for the backup to finish. Jar files are unlogged but the 
+            // changes to the  references to the jar file in the catalogs 
+            // is logged. A consistent backup can not be made when jar file 
+            // is being added.
+
+            tran.blockBackup(true);
 
 			StorageFile directory = file.getParentDir();
             if (!directory.exists())
@@ -152,15 +156,19 @@
 			
 		ContextManager cm = ContextService.getFactory().getCurrentContextManager();
 
-		Transaction tran = 
-            factory.getRawStoreFactory().findUserTransaction(
-                cm, AccessFactoryGlobals.USER_TRANS_NAME);
-
-        // Prevent backup operation when a jar file is being removed
-        // by setting the transaction into a backup blocking state.
-        // If backup is already in progress this call will wait 
-        // for the backup to finish.
-        ((RawTransaction)tran).setBackupBlockingState(true);
+        RawTransaction tran = 
+            factory.getRawStoreFactory().getXactFactory().findUserTransaction(
+                        factory.getRawStoreFactory(), 
+                        cm, 
+                        AccessFactoryGlobals.USER_TRANS_NAME);
+                    
+        // Block the backup, If backup is already in progress wait 
+        // for the backup to finish. Jar files are unlogged but the 
+        // changes to the  references to the jar file in the catalogs 
+        // is logged. A consistent backup can not be made when jar file 
+        // is being removed.
+
+        tran.blockBackup(true);
 
 		tran.logAndDo(new RemoveFileOperation(name, currentGenerationId, purgeOnCommit));
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java Thu Jan 19 08:32:08 2006
@@ -245,7 +245,7 @@
 	// true, if the transaction executed some operations(like unlogged
 	// operations) that block the  online backup to prevent inconsistent
 	// backup copy.
-	private boolean inBackupBlockingState;
+	private boolean backupBlocked;
 
 
 	/*
@@ -285,7 +285,7 @@
 
 		setIdleState();
 
-		inBackupBlockingState = false; 
+		backupBlocked = false; 
 
         /*
         System.out.println("Xact.constructor: readonly = " + this.readOnly +
@@ -1910,9 +1910,9 @@
 		setIdleState();
 
 		// any backup blocking operations (like unlogged ops) in this 
-		// transaction are done with post commit/abort work by this time, 
-		// change the transaction to unblock the backup state.
-		setUnblockBackupState();
+		// transaction are done with post commit/abort work by now, 
+		// unblock the backup.
+		unblockBackup();
 		inComplete = null;
 	}
 
@@ -2318,38 +2318,47 @@
 		postCompleteMode = true;
 	}
 
+
     /*
-     * Try setting the transaction to be in backup blocking state.
+     * Make the transaction block the online backup.
      *
      * @param wait if <tt>true</tt>, waits until the transaction
-     *             can be set into backup blocking state.
-     *
+     *             can block the backup.
+     * @return     <tt>true</tt> if the transaction  blocked the  
+     *             backup.  <tt>false</tt> otherwise.
      * @exception StandardException if interrupted while waiting 
-     *            for backup to complete to set the transaction into
-     *            backup blocking state.
+     *            for the backup in progress to complete.
      */
-    public boolean setBackupBlockingState(boolean wait) 
+    public boolean blockBackup(boolean wait) 
         throws StandardException
     {
-		if (!inBackupBlockingState)
-        {
-			inBackupBlockingState = 
-                xactFactory.canStartBackupBlockingOperation(wait);
+		if (!backupBlocked) {
+			backupBlocked = xactFactory.blockBackup(wait);
         }
 
-		return inBackupBlockingState;
+		return backupBlocked;
 	}
 	
 	/*
 	 * Unblock the backup, if it was blocked by some operation in 
-	 * this transaction. Unbloking is done at commit/abort of this 
+	 * this transaction. Unblocking is done at commit/abort of this 
 	 * transaction.
 	 */
-	private void setUnblockBackupState() {
-		if (inBackupBlockingState)
-			xactFactory.backupBlockingOperationFinished();	
-		inBackupBlockingState = false;
+	private void unblockBackup() {
+		if (backupBlocked)
+			xactFactory.unblockBackup();	
+		backupBlocked = false;
 	}
+
+
+    /**
+     * Check if the transaction is blocking the backup ?
+     * @return <tt> true </tt> if this transaction is 
+     *         blocking the backup, otherwise <tt> false </tt>
+     */
+    public boolean isBlockingBackup() {
+        return backupBlocked;
+    }
 
 	/*
 	** Lock escalation related

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java Thu Jan 19 08:32:08 2006
@@ -968,17 +968,18 @@
 
 
     /**
-     * Checks if a backup blocking operation can be started.
-     *
-     * @param wait if <tt>true</tt>, waits until a backup blocking 
-     *             operation can be started. 
-     *
-     * @return     <tt>true</tt> if backup blocking operations can be started.
+     * Block the online backup. Backup needs to be blocked while 
+     * executing any unlogged operations or any opearation that 
+     * prevents from  making a consistent backup.
+     * 
+     * @param wait if <tt>true</tt>, waits until the backup 
+     *             is blocked. 
+     * @return     <tt>true</tt> if backup is blocked.
      *			   <tt>false</tt> otherwise.
-     * @exception StandardException if interrupted while waiting for backup 
-     *             to complete.
+     * @exception StandardException if interrupted while waiting for a 
+     *           backup  to complete.
      */
-	protected boolean canStartBackupBlockingOperation(boolean wait)
+	protected boolean blockBackup(boolean wait)
         throws StandardException 
 	{
 		synchronized(backupSemaphore) {
@@ -1007,9 +1008,9 @@
 
 
 	/**
-	 * Mark that a backup blocking operation finished. 
+     * Unblock the backup, a backup blocking operation finished. 
 	 */
-	protected void backupBlockingOperationFinished()
+	protected void unblockBackup()
 	{
 		synchronized(backupSemaphore) {
 			if (SanityManager.DEBUG)
@@ -1027,7 +1028,7 @@
 
 	/**
 	 * Checks if there are any backup blocking operations in progress and 
-	 * stops new ones from starting until the backup is finished. 
+	 * prevents new ones from starting until the backup is finished. 
 	 * If backup blocking operations are in progress and  <code> wait </code>
 	 * parameter value is <tt>true</tt>, then it will wait for the current 
 	 * backup blocking operations to finish. 
@@ -1042,7 +1043,7 @@
 	 *             <tt>false</tt> otherwise.
 	 * @exception StandardException if interrupted or a runtime exception occurs
 	 */
-	public boolean stopBackupBlockingOperations(boolean wait) 
+	public boolean blockBackupBlockingOperations(boolean wait) 
 		throws StandardException 
 	{
 		synchronized(backupSemaphore) {
@@ -1099,7 +1100,7 @@
 	/**
 	 * Backup completed. Allow backup blocking operations. 
 	 */
-	public void backupFinished()
+	public void unblockBackupBlockingOperations()
 	{
 		synchronized(backupSemaphore) {
 			inBackup = false;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties Thu Jan 19 08:32:08 2006
@@ -188,8 +188,8 @@
 XSRS7.S=Backup caught unexpected exception.
 XSRS8.S=Log Device can only be set during database creation time, it cannot be changed on the fly.
 XSRS9.S=Record {0} no longer exists
-XSRSA.S=Cannot backup the database when backup blocking unlogged operations are pending. Please commit the transactions with backup blocking operations or use the backup procedure with option to wait for them to complete. 
-XSRSB.S=Backup operation can not be performed in an active transaction. Please use a new transaction to execute backup procedures.
+XSRSA.S=Cannot backup the database when unlogged operations are uncommitted. Please commit the transactions with backup blocking operations. 
+XSRSB.S=Backup cannot be performed in a transaction with uncommitted unlogged operations.
 
 # java/com/ibm/db2j/impl/Database/Storage/RawStore/Log/Generic
 # statement errors.

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest1.out
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest1.out?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest1.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest1.out Thu Jan 19 08:32:08 2006
@@ -1,5 +1,6 @@
 Begin Online Backup Test1
-A Transaction with Unlogged Operation Started
+First Transaction with Unlogged Operation Started
+Second Transaction with Unlogged Operation Started
 BACKUP STARTED
 database shutdown properly
 Restored From the Backup

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest3.out
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest3.out?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest3.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest3.out Thu Jan 19 08:32:08 2006
@@ -1,7 +1,7 @@
 Begin Online Backup Test3
 Initial Setup Complete
 Begin Install Jar Test
-ERROR XSRSA: Cannot backup the database when backup blocking unlogged operations are pending. Please commit the transactions with backup blocking operations or use the backup procedure with option to wait for them to complete. 
+ERROR XSRSA: Cannot backup the database when unlogged operations are uncommitted. Please commit the transactions with backup blocking operations. 
 Backup-1 Started
 The transaction that was blocking the backup has ended
 Backup-1 Completed
@@ -19,7 +19,7 @@
 database shutdown properly
 End Of Install Jar Test.
 Begin Remove Jar Test
-ERROR XSRSA: Cannot backup the database when backup blocking unlogged operations are pending. Please commit the transactions with backup blocking operations or use the backup procedure with option to wait for them to complete. 
+ERROR XSRSA: Cannot backup the database when unlogged operations are uncommitted. Please commit the transactions with backup blocking operations. 
 Backup-3 Started
 The transaction that was blocking the backup has ended
 Backup-3 Completed

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/onlineBackupTest2.out
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/onlineBackupTest2.out?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/onlineBackupTest2.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/onlineBackupTest2.out Thu Jan 19 08:32:08 2006
@@ -26,59 +26,51 @@
 ij(C1)> insert into t1 values(2) ;
 1 row inserted/updated/deleted
 ij(C1)> commit ;
-ij(C1)> -- make sure backup procedure are not allowed in an non-idle transaction.
-insert into t1 values(3) ;
+ij(C1)> -- make sure backup calls are not allowed in a transaction that
+-- has executed unlogged operations before the backup calls. 
+insert into t1 values(3);
 1 row inserted/updated/deleted
+ij(C1)> create index idx1 on t1(a);
+0 rows inserted/updated/deleted
 ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE('extinout/mybackup') ;
-ERROR XSRSB: Backup operation can not be performed in an active transaction. Please use a new transaction to execute backup procedures.
-ij(C1)> commit;
-ij(C1)> select * from t1 ;
-A          
------------
-1          
-2          
-3          
-ij(C1)> call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup' , 1) ;
-ERROR XSRSB: Backup operation can not be performed in an active transaction. Please use a new transaction to execute backup procedures.
-ij(C1)> rollback;
-ij(C1)> select * from t1 ;
+ERROR XSRSB: Backup cannot be performed in a transaction with uncommitted unlogged operations.
+ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT('extinout/mybackup') ;
+ERROR XSRSB: Backup cannot be performed in a transaction with uncommitted unlogged operations.
+ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
+                                              'extinout/mybackup', 1);
+ERROR XSRSB: Backup cannot be performed in a transaction with uncommitted unlogged operations.
+ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT(
+                                              'extinout/mybackup', 1);
+ERROR XSRSB: Backup cannot be performed in a transaction with uncommitted unlogged operations.
+ij(C1)> --backup failures should not rollback/commit the transaction. 
+select * from t1 ;
 A          
 -----------
 1          
 2          
 3          
-ij(C1)> call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup' , 0) ;
-ERROR XSRSB: Backup operation can not be performed in an active transaction. Please use a new transaction to execute backup procedures.
-ij(C1)> commit;
-ij(C1)> update t1 set a = a + 1 ;
-3 rows inserted/updated/deleted
-ij(C1)> call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
-                                              'extinout/mybackup', 1, 1);
-ERROR XSRSB: Backup operation can not be performed in an active transaction. Please use a new transaction to execute backup procedures.
+ij(C1)> insert into t1 values(4) ;
+1 row inserted/updated/deleted
 ij(C1)> commit;
-ij(C1)> update t1 set a = a - 1 ;
-3 rows inserted/updated/deleted
-ij(C1)> call SYSCS_UTIL.SYSCS_DISABLE_LOG_ARCHIVE_MODE(1);
-ERROR XSRSB: Backup operation can not be performed in an active transaction. Please use a new transaction to execute backup procedures.
+ij(C1)> drop index idx1;
+0 rows inserted/updated/deleted
 ij(C1)> commit;
 ij(C1)> --- make sure backup calls can be run one after another.
-insert into t1 values(4) ;
-1 row inserted/updated/deleted
-ij(C1)> insert into t1 values(5) ;
+insert into t1 values(5) ;
 1 row inserted/updated/deleted
-ij(C1)> rollback;
 ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE('extinout/mybackup') ;
 0 rows inserted/updated/deleted
-ij(C1)> call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup' , 1) ;
+ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT('extinout/mybackup');
 0 rows inserted/updated/deleted
 ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
                                            'extinout/mybackup', 1);
 0 rows inserted/updated/deleted
-ij(C1)> call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
-                                               'extinout/mybackup', 1, 1);
+ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT(
+                                               'extinout/mybackup', 1);
 0 rows inserted/updated/deleted
 ij(C1)> call SYSCS_UTIL.SYSCS_DISABLE_LOG_ARCHIVE_MODE(1);
 0 rows inserted/updated/deleted
+ij(C1)> commit;
 ij(C1)> -- make sure backup is not allowed when non-logged 
 -- operations are pending
 connect 'wombat' as c2 ;
@@ -89,12 +81,12 @@
 0 rows inserted/updated/deleted
 ij(C2)> set connection c1 ;
 ij(C1)> -- following two backup calls should fail , because they are not waiting
--- for the unlogged index creation to commit/rollback.
-call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup' , 0) ;
-ERROR XSRSA: Cannot backup the database when backup blocking unlogged operations are pending. Please commit the transactions with backup blocking operations or use the backup procedure with option to wait for them to complete. 
-ij(C1)> call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
-                                               'extinout/mybackup', 1, 0);
-ERROR XSRSA: Cannot backup the database when backup blocking unlogged operations are pending. Please commit the transactions with backup blocking operations or use the backup procedure with option to wait for them to complete. 
+-- for the unlogged index creation in anothere transaction to commit/rollback.
+call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT('extinout/mybackup') ;
+ERROR XSRSA: Cannot backup the database when unlogged operations are uncommitted. Please commit the transactions with backup blocking operations. 
+ij(C1)> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT(
+                                               'extinout/mybackup', 1);
+ERROR XSRSA: Cannot backup the database when unlogged operations are uncommitted. Please commit the transactions with backup blocking operations. 
 ij(C1)> set connection c2;
 ij(C2)> rollback ;
 ij(C2)> -- make sure backup call waits, if wait parameter value is non-zero or 
@@ -123,8 +115,8 @@
 1                                                                                                                               
 --------------------------------------------------------------------------------------------------------------------------------
 false                                                                                                                           
-ij(C1)> async bthread1 'call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE(
-                                    ''extinout/ulbackup1'' , 1)' ;
+ij(C1)> async bthread1 'call SYSCS_UTIL.SYSCS_BACKUP_DATABASE(
+                                    ''extinout/ulbackup1'')' ;
 ij(C1)> set connection c2;
 ij(C2)> -- sleep for a while for the backup thread to 
 -- really get into the wait state
@@ -200,8 +192,8 @@
 --------------------------------------------------------------------------------------------------------------------------------
 false                                                                                                                           
 ij(C1)> async bthread1 
-  'call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
-                                    ''extinout/ulbackup3'' , 1, 1)' ;
+  'call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
+                                    ''extinout/ulbackup3'' , 1)' ;
 ij(C1)> set connection c2;
 ij(C2)> -- sleep for a while for the backup thread to 
 -- really get into the wait state

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1.java Thu Jan 19 08:32:08 2006
@@ -40,7 +40,9 @@
 public class OnlineBackupTest1 {
 
 	private static final String TEST_DATABASE_NAME = "wombat" ;
-	private static final String TEST_TABLE_NAME =    "emp";
+	private static final String TEST_TABLE_NAME   =    "emp";
+    private static final String TEST_TABLE_NAME_1 =    "emp_1";
+    private static final String TEST_TABLE_NAME_2 =    "emp_2";
 
 	public static void main(String[] argv) throws Throwable {
 		
@@ -67,10 +69,23 @@
 		Connection conn = ij.startJBMS();
 		conn.setAutoCommit(false);
 		DatabaseActions dbActions = new DatabaseActions(conn);
-		//create the test  table. 
+		//create the test  tables. 
 		dbActions.createTable(TEST_TABLE_NAME);
-		dbActions.startUnloggedAction(TEST_TABLE_NAME);
-		logMessage("A Transaction with Unlogged Operation Started");
+        dbActions.createTable(TEST_TABLE_NAME_1);
+        dbActions.createTable(TEST_TABLE_NAME_2);
+
+        // start first unlogged operation
+		dbActions.startUnloggedAction(TEST_TABLE_NAME_1);
+		logMessage("First Transaction with Unlogged Operation Started");
+
+        // start second unlogged opearation
+        Connection conn1 = ij.startJBMS();
+		conn1.setAutoCommit(false);
+		DatabaseActions dbActions1 = new DatabaseActions(conn1);
+		dbActions1.startUnloggedAction(TEST_TABLE_NAME_2);
+		logMessage("Second Transaction with Unlogged Operation Started");
+        
+
 		// start a  thread to perform online backup
 		OnlineBackup backup = new OnlineBackup(TEST_DATABASE_NAME);
 		Thread backupThread = new Thread(backup, "BACKUP");
@@ -111,8 +126,10 @@
 			logMessage("Backup is not waiting for unlogged actions to commit");
 
 		// end the unlogged work transaction.
-		dbActions.endUnloggedAction(TEST_TABLE_NAME);
-		
+		dbActions.endUnloggedAction(TEST_TABLE_NAME_1);
+        // end the unlogged work transaction.
+		dbActions1.endUnloggedAction(TEST_TABLE_NAME_2);
+        
 		backup.waitForBackupToEnd();
 		backupThread.join();
 		dmlActions.stopActivity();
@@ -122,6 +139,7 @@
         
         // close the connections.
         conn.close();
+        conn1.close();
         dmlConn.close();
         ddlConn.close() ;
 
@@ -147,7 +165,11 @@
         Connection conn = getConnection();
 		Statement stmt = conn.createStatement();
 		stmt.execute("values SYSCS_UTIL.SYSCS_CHECK_TABLE('APP',  'EMP')");
-		//TO DO : Consistency check all the tables including the system tables. 
+        //check the data in the EMP table.
+        DatabaseActions dbActions = new DatabaseActions(conn);
+        dbActions.select(TEST_TABLE_NAME);
+        dbActions.select(TEST_TABLE_NAME_1);
+        dbActions.select(TEST_TABLE_NAME_2);
 		conn.close();
 
 	}
@@ -273,10 +295,9 @@
 		 */
 		void performDmlActions() throws SQLException {
 			
-			while(stopActivity) {
+			while(!stopActivity) {
 				insert(TEST_TABLE_NAME, 100, COMMIT, 10);
 				insert(TEST_TABLE_NAME, 100, ROLLBACK, 10);
-				update(TEST_TABLE_NAME, 50, COMMIT, 10);
 				update(TEST_TABLE_NAME, 50, ROLLBACK, 10);
 				select(TEST_TABLE_NAME);
 			}
@@ -294,9 +315,10 @@
 			insert(tableName, 100, COMMIT, 10);
 			// execute a unlogged database operation
 			Statement s = conn.createStatement();
-			// index creation does not log the index entries 
-			s.executeUpdate("create index " + tableName + "_name_idx on " + 
-							TEST_TABLE_NAME + "(name) ");
+			
+            // index creation does not log the index entries 
+            s.executeUpdate("create index " + tableName + "_name_idx on " + 
+                            tableName + "(name) ");
 			s.close();
 		}
 
@@ -307,11 +329,12 @@
 		 * @exception SQLException if any database exception occurs.
 		 */
 		void endUnloggedAction(String tableName) throws SQLException {
+            // insert some rows, insert should be successful even if
+            // backup is blocking for uncommitted unlogged operations. 
 			insert(tableName, 1000, OPENTX, 10);
 			conn.commit();
 		}
 
-
 				
 		/**
 		 * Create and Drop some tables.
@@ -401,10 +424,10 @@
 		
 		/**
 		 * update some rows in the table.
-		 * @param  tableName  name of the table that rows are inserted.
-		 * @param  rowCount   Number of rows to Insert.
+		 * @param  tableName  name of the table that rows are updates.
+		 * @param  rowCount   Number of rows to update.
 		 * @param  txStaus    Transacton status commit/rollback/open.
-		 * @param  commitCount After how many inserts commit/rollbacku should
+		 * @param  commitCount After how many updates commit/rollback should
 		 *                      happen.
 		 * @exception SQLException if any database exception occurs.
 		 */
@@ -414,11 +437,10 @@
 		{
 
 			PreparedStatement ps = conn.prepareStatement("update " + tableName + 
-														 " SET salary=? where id=?");
+								 " SET name = ?  where id=?");
 		
 			for (int i = 0; i < rowCount; i++) {
-
-				ps.setFloat(1, (float)(i * 2000 * 0.08));
+                ps.setString(1 ,  "moonwalker" + i);
 				ps.setInt(2, i); // ID
 				ps.executeUpdate();
 				if ((i % commitCount) == 0)
@@ -447,7 +469,7 @@
 			{
 				int tid = rs.getInt(1);
 				String name = rs.getString(2);
-				if(name.equals("skywalker" + id) && tid!= id)
+ 				if(name.equals("skywalker" + id) && tid!= id)
 				{
 					logMessage("DATA IN THE TABLE IS NOT AS EXPECTED");
 					logMessage("Got :ID=" +  tid + " Name=:" + name);
@@ -457,9 +479,10 @@
 				id++;
 				count++;
 			}
-
+            
 			rs.close();
 			s.close();
+            conn.commit();
 		}
 
 		/* 

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest3.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest3.java?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest3.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest3.java Thu Jan 19 08:32:08 2006
@@ -39,28 +39,28 @@
 
 public class OnlineBackupTest3 {
 
-	private static final String TEST_DATABASE_NAME = "wombat" ;
+    private static final String TEST_DATABASE_NAME = "wombat" ;
+
+    public static void main(String[] argv) throws Throwable {
 
-	public static void main(String[] argv) throws Throwable {
-		
         OnlineBackupTest3 test = new OnlineBackupTest3();
-   		ij.getPropertyArg(argv); 
+        ij.getPropertyArg(argv); 
 
         try {
             test.runTest();
         }
         catch (SQLException sqle) {
-			dumpSQLException(sqle);
-		} 
+            dumpSQLException(sqle);
+        } 
     }
 
 
-	/*
-	 * Test online backup with unlogged jar operations running in parallel. 
-	 */
-	private void runTest() throws SQLException, Exception {
-		logMessage("Begin Online Backup Test3");
-		Connection conn = ij.startJBMS();
+    /*
+     * Test online backup with unlogged jar operations running in parallel. 
+     */
+    private void runTest() throws SQLException, Exception {
+        logMessage("Begin Online Backup Test3");
+        Connection conn = ij.startJBMS();
         conn.setAutoCommit(false);
         Statement stmt = conn.createStatement();
         stmt.execute("create table t1(a int ) ");
@@ -94,63 +94,62 @@
         // online backup running in parallel.
         removeJarTest();
 
-		logMessage("End Online Backup Test3");
-	}
+        logMessage("End Online Backup Test3");
+    }
+
 
-		
-	/**
-	 * Shutdown the datbase
-	 * @param  dbName  Name of the database to shutdown.
-	 */
-	void shutdown(String dbName) {
-
-		try{
-			//shutdown
-			if(TestUtil.HAVE_DRIVER_CLASS)
-				DriverManager.getConnection("jdbc:derby:" + dbName + ";shutdown=true");
-			else 
-				TestUtil.shutdownUsingDataSource(dbName);
-		}catch(SQLException se){
-			if (se.getSQLState() != null && se.getSQLState().equals("08006"))
-				System.out.println("database shutdown properly");
-			else
-				dumpSQLException(se);
-		}
-	}
+    /**
+     * Shutdown the datbase
+     * @param  dbName  Name of the database to shutdown.
+     */
+    void shutdown(String dbName) {
+
+        try{
+            //shutdown
+            if(TestUtil.HAVE_DRIVER_CLASS)
+                DriverManager.getConnection("jdbc:derby:" + dbName + ";shutdown=true");
+            else 
+                TestUtil.shutdownUsingDataSource(dbName);
+        }catch(SQLException se){
+            if (se.getSQLState() != null && se.getSQLState().equals("08006"))
+                System.out.println("database shutdown properly");
+            else
+                dumpSQLException(se);
+        }
+    }
 
     /*
      * get connection to the test database
      */
     Connection getConnection() throws SQLException 
     {
-    	Connection conn;
-    	if(TestUtil.HAVE_DRIVER_CLASS)
-			conn = DriverManager.getConnection("jdbc:derby:" + TEST_DATABASE_NAME );
-    	else {
-	    	Properties prop = new Properties();
-	        prop.setProperty("databaseName", TEST_DATABASE_NAME);
-	        conn = TestUtil.getDataSourceConnection(prop);
-    	}
+        Connection conn;
+        if(TestUtil.HAVE_DRIVER_CLASS)
+            conn = DriverManager.getConnection("jdbc:derby:" + TEST_DATABASE_NAME );
+        else {
+            Properties prop = new Properties();
+            prop.setProperty("databaseName", TEST_DATABASE_NAME);
+            conn = TestUtil.getDataSourceConnection(prop);
+        }
         return conn;
     }
 
 
-	/**
-	 * Write message to the standard output.
-	 */
-	void logMessage(String   str)	{
-			System.out.println(str);
-	}
-
-	
-	/**
-	 * dump the SQLException to the standard output.
-	 */
-	static private void dumpSQLException(SQLException sqle) {
-		
-		org.apache.derby.tools.JDBCDisplayUtil.	ShowSQLException(System.out, sqle);
-		sqle.printStackTrace(System.out);
-	}
+    /**
+     * Write message to the standard output.
+     */
+    void logMessage(String   str){
+        System.out.println(str);
+    }
+
+    /**
+     * dump the SQLException to the standard output.
+     */
+    static private void dumpSQLException(SQLException sqle) {
+
+        org.apache.derby.tools.JDBCDisplayUtil.ShowSQLException(System.out, sqle);
+        sqle.printStackTrace(System.out);
+    }
 
     
     private int countRows(Connection conn, 
@@ -183,54 +182,52 @@
            "call sqlj.install_jar('extin/brtestjar.jar', 'math_routines', 0)");
         
         try {
-            // followng backup call should because jar operation is pending 
+            // followng backup call should fail because jar operation is pending 
            conn2_stmt.execute(
-            "call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup', 0)");
+            "call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT('extinout/mybackup')");
         } catch (SQLException sqle) {
             //above statement should have failed. 
-            org.apache.derby.tools.JDBCDisplayUtil.	ShowSQLException(System.out, sqle);
+            org.apache.derby.tools.JDBCDisplayUtil.ShowSQLException(System.out, sqle);
         }
 
-        // invoke backup in another thread, should block for the above install jar 
-        // operation for 'brtestjar.jar to commit.
+        // invoke backup in another thread, it should block for the above install jar 
+        // operation to install  'brtestjar.jar to commit.
         
         // start a  thread to perform online backup
-		OnlineBackup backup = new OnlineBackup(TEST_DATABASE_NAME);
-		Thread backupThread = new Thread(backup, "BACKUP1");
-		backupThread.start();	
-		// wait for the backup to start
-		backup.waitForBackupToBegin();
-		logMessage("Backup-1 Started");
+        OnlineBackup backup = new OnlineBackup(TEST_DATABASE_NAME);
+        Thread backupThread = new Thread(backup, "BACKUP1");
+        backupThread.start();
+        // wait for the backup to start
+        backup.waitForBackupToBegin();
+        logMessage("Backup-1 Started");
 
-        // sleep for few seconds just to make sure backup thread is actually
-		// gone to a wait state for unlogged actions to commit.
-		java.lang.Thread.sleep(1000);
-			
-		// backup should not even start doing real work before the
-		// unlogged transaction is commited
-		if(!backup.isRunning())
-			logMessage("Backup is not waiting for unlogged " +  
+        // sleep for few seconds just to make sure backup thread has actually
+        // gone into a wait state for unlogged actions to commit.
+        java.lang.Thread.sleep(1000);
+        
+        // backup should not even start doing real work before the
+        // unlogged transaction is commited
+        if(!backup.isRunning())
+            logMessage("Backup is not waiting for unlogged " +  
                        "install jar action to commit");
 
         //insert some rows that should appear in the backup.
         conn1_stmt.execute("insert into t1 values(3)");
         conn1_stmt.execute("insert into t1 values(4)");
-
+        conn1_stmt.execute("insert into t1 values(5)");
         
         // set the database class with both the jars  installed above.
         conn1_stmt.execute("CALL SYSCS_UTIL.SYSCS_SET_DATABASE_PROPERTY( " + 
                            "'derby.database.classpath', " + 
                            "'APP.math_routines') " ) ;
 
-        //Now commit the jar operation in connection1 for backup to proceed. 
-        conn1_stmt.execute("insert into t1 values(5)");
         //commit the transaction with jar opearation that is blocking the backup.
         conn1.commit();
         logMessage("The transaction that was blocking the backup has ended");
 
         // wait for backup to finish. 
         backup.waitForBackupToEnd();
-		backupThread.join();
+        backupThread.join();
         logMessage("Backup-1 Completed");
         
         // Case : jar op should block if backup is in progress
@@ -241,19 +238,19 @@
         // start a  thread to perform online backup
         backup = new OnlineBackup(TEST_DATABASE_NAME);
         backupThread = new Thread(backup, "BACKUP2");
-		backupThread.start();	
-		// wait for the backup to start
-		backup.waitForBackupToBegin();
-		logMessage("Backup-2 Started");
+        backupThread.start();
+        // wait for the backup to start
+        backup.waitForBackupToBegin();
+        logMessage("Backup-2 Started");
 
         // sleep for few seconds just to make sure backup thread is actually
-		// gone to a wait state for unlogged actions to commit.
-		java.lang.Thread.sleep(1000);
-			
-		// backup should not even start doing real work before the
-		// unlogged transaction is commited
-		if(!backup.isRunning())
-			logMessage("Backup is not waiting for unlogged " +  
+        // gone to a wait state for unlogged actions to commit.
+        java.lang.Thread.sleep(1000);
+
+        // backup should not even start doing real work before the
+        // unlogged transaction is commited
+        if(!backup.isRunning())
+            logMessage("Backup is not waiting for unlogged " +  
                        "index action to commit");
 
 
@@ -278,7 +275,7 @@
 
         // wait for backup to finish. 
         backup.waitForBackupToEnd();
-		backupThread.join();
+        backupThread.join();
         logMessage("Backup-2 Completed");
         
         // wait for customer app jar installation to finish now. 
@@ -308,10 +305,10 @@
         conn2.close();
         
         //shutdown the test db 
-		shutdown(TEST_DATABASE_NAME);
-		// restore the database from the backup and run some checks 
-		backup.restoreFromBackup();
-		logMessage("Restored From the Backup");
+        shutdown(TEST_DATABASE_NAME);
+        // restore the database from the backup and run some checks 
+        backup.restoreFromBackup();
+        logMessage("Restored From the Backup");
         Connection conn = getConnection();
         Statement stmt = conn.createStatement();
         logMessage("No of rows in table t1: " + countRows(conn, "T1"));
@@ -336,9 +333,9 @@
         stmt.close();
         conn.close();
 
-		//shutdown the test db 
-		shutdown(TEST_DATABASE_NAME);
-		logMessage("End Of Install Jar Test.");
+        //shutdown the test db 
+        shutdown(TEST_DATABASE_NAME);
+        logMessage("End Of Install Jar Test.");
 
     }
 
@@ -362,7 +359,7 @@
         }
 
         // remove both the jars from the class path , 
-        // so that we can remove them. 
+        // so that we can remove them from the database. 
         conn1_stmt.execute("CALL SYSCS_UTIL.SYSCS_SET_DATABASE_PROPERTY( " + 
                         "'derby.database.classpath', '')") ;
         conn1.commit();
@@ -370,35 +367,44 @@
         conn1_stmt.execute(
            "call sqlj.remove_jar('APP.math_routines', 0)");
         
+        // Case 0: backup call that is not waiting for unlogged 
+        // opereation to complete should fail when a remove jar 
+        // is not ended when backup started. 
+
         try {
-            // followng backup call should because remove 
+            // followng backup call should fail because remove 
             // jar operation is pending 
            conn2_stmt.execute(
-            "call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup', 0)");
+            "call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT('extinout/mybackup')");
         } catch (SQLException sqle) {
             //above statement should have failed. 
-            org.apache.derby.tools.JDBCDisplayUtil.	ShowSQLException(System.out, sqle);
+            org.apache.derby.tools.JDBCDisplayUtil.ShowSQLException(System.out, sqle);
         }
-            
+
+
+                
+        // Case 1: backup should block because when a remove jar
+        // is not ended when backup started. 
+
         // invoke backup in another thread, should block for 
         // the above remove jar  to commit.
         
         // start a  thread to perform online backup
-		OnlineBackup backup = new OnlineBackup(TEST_DATABASE_NAME);
-		Thread backupThread = new Thread(backup, "BACKUP3");
-		backupThread.start();	
-		// wait for the backup to start
-		backup.waitForBackupToBegin();
-		logMessage("Backup-3 Started");
+        OnlineBackup backup = new OnlineBackup(TEST_DATABASE_NAME);
+        Thread backupThread = new Thread(backup, "BACKUP3");
+        backupThread.start();
+        // wait for the backup to start
+        backup.waitForBackupToBegin();
+        logMessage("Backup-3 Started");
 
         // sleep for few seconds just to make sure backup thread is actually
-		// gone to a wait state for unlogged actions to commit.
-		java.lang.Thread.sleep(1000);
-			
-		// backup should not even start doing real work before the
-		// unlogged transaction is commited
-		if(!backup.isRunning())
-			logMessage("Backup is not waiting for unlogged " +  
+        // gone to a wait state for unlogged actions to commit.
+        java.lang.Thread.sleep(1000);
+
+        // backup should not even start doing real work before the
+        // unlogged transaction is commited
+        if(!backup.isRunning())
+            logMessage("Backup is not waiting for unlogged " +  
                        "remove jar action to commit");
 
         //insert some rows that should appear in the backup.
@@ -411,7 +417,7 @@
         
         // wait for backup to finish. 
         backup.waitForBackupToEnd();
-		backupThread.join();
+        backupThread.join();
 
         logMessage("Backup-3 Completed");
         
@@ -423,19 +429,19 @@
         // start a  thread to perform online backup
         backup = new OnlineBackup(TEST_DATABASE_NAME);
         backupThread = new Thread(backup, "BACKUP4");
-		backupThread.start();	
-		// wait for the backup to start
-		backup.waitForBackupToBegin();
-		logMessage("Backup-4 Started");
+        backupThread.start();
+        // wait for the backup to start
+        backup.waitForBackupToBegin();
+        logMessage("Backup-4 Started");
 
         // sleep for few seconds just to make sure backup thread is actually
-		// gone to a wait state for unlogged actions to commit.
-		java.lang.Thread.sleep(1000);
-			
-		// backup should not even start doing real work before the
-		// unlogged transaction is commited
-		if(!backup.isRunning())
-			logMessage("Backup is not waiting for unlogged " +  
+        // gone to a wait state for unlogged actions to commit.
+        java.lang.Thread.sleep(1000);
+
+        // backup should not even start doing real work before the
+        // unlogged transaction is commited
+        if(!backup.isRunning())
+            logMessage("Backup is not waiting for unlogged " +  
                        "index action to commit");
 
 
@@ -459,10 +465,10 @@
         logMessage("The transaction that was blocking the backup has ended");
         // wait for backup to finish. 
         backup.waitForBackupToEnd();
-		backupThread.join();
+        backupThread.join();
         logMessage("Backup-4 Completed");
 
-        // wait for customer app jar installation to finish now. 
+        // wait for customer app jar removal to finish now. 
         asyncJarActionThread.join();
         logMessage("obtest_customer.jar remove is complete");
         
@@ -480,10 +486,10 @@
         conn2.close();
         
         //shutdown the test db 
-		shutdown(TEST_DATABASE_NAME);
-		// restore the database from the backup and run some checks 
-		backup.restoreFromBackup();
-		logMessage("Restored From the Backup");
+        shutdown(TEST_DATABASE_NAME);
+        // restore the database from the backup and run some checks 
+        backup.restoreFromBackup();
+        logMessage("Restored From the Backup");
         Connection conn = getConnection();
         Statement stmt = conn.createStatement();
         logMessage("No of rows in table t1: " + countRows(conn, "T1"));
@@ -493,22 +499,23 @@
         // check if the jar removal was successful.
         // APP.math_routines should not be in backup.
         try {
-            // set the database class with both the jars  installed above.
+            // set the database class path with the jar removed above, 
+            // it should fail.
             stmt.execute("CALL SYSCS_UTIL.SYSCS_SET_DATABASE_PROPERTY( " + 
                            "'derby.database.classpath', " + 
                            "'APP.math_routines') " ) ;
         }catch (SQLException sqle) {
             //above statement should have failed. 
-            org.apache.derby.tools.JDBCDisplayUtil.	ShowSQLException(System.out, sqle);
+            org.apache.derby.tools.JDBCDisplayUtil.ShowSQLException(System.out, sqle);
         }
         
 
         stmt.close();
         conn.close();
 
-		//shutdown the test db 
-		shutdown(TEST_DATABASE_NAME);
-		logMessage("End Of Remove Jar Test.");
+        //shutdown the test db 
+        shutdown(TEST_DATABASE_NAME);
+        logMessage("End Of Remove Jar Test.");
 
     }
 
@@ -537,7 +544,7 @@
                 conn.commit();
             } catch (SQLException sqle) {
                 org.apache.derby.tools.JDBCDisplayUtil.ShowSQLException(System.out, sqle);
-				sqle.printStackTrace(System.out);
+                sqle.printStackTrace(System.out);
             }
             aStatement = null;
         }

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/onlineBackupTest2.sql
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/onlineBackupTest2.sql?rev=370530&r1=370529&r2=370530&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/onlineBackupTest2.sql (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/onlineBackupTest2.sql Thu Jan 19 08:32:08 2006
@@ -25,35 +25,32 @@
 insert into t1 values(1) ;
 insert into t1 values(2) ; 
 commit ;
--- make sure backup procedure are not allowed in an non-idle transaction.
-insert into t1 values(3) ;
+-- make sure backup calls are not allowed in a transaction that
+-- has executed unlogged operations before the backup calls. 
+insert into t1 values(3); 
+create index idx1 on t1(a);
 call SYSCS_UTIL.SYSCS_BACKUP_DATABASE('extinout/mybackup') ;
-commit;
-select * from t1 ;
-call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup' , 1) ;
-rollback;
+call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT('extinout/mybackup') ;
+call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
+                                              'extinout/mybackup', 1);
+call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT(
+                                              'extinout/mybackup', 1);
+--backup failures should not rollback/commit the transaction. 
 select * from t1 ;
-call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup' , 0) ;
+insert into t1 values(4) ;
 commit;
-update t1 set a = a + 1 ;
-call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
-                                              'extinout/mybackup', 1, 1);
-commit;
-update t1 set a = a - 1 ;
-call SYSCS_UTIL.SYSCS_DISABLE_LOG_ARCHIVE_MODE(1);
+drop index idx1;
 commit;
 --- make sure backup calls can be run one after another.
-insert into t1 values(4) ;
 insert into t1 values(5) ;
-rollback;
 call SYSCS_UTIL.SYSCS_BACKUP_DATABASE('extinout/mybackup') ;
-call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup' , 1) ;
+call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT('extinout/mybackup');
 call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
                                            'extinout/mybackup', 1);
-call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
-                                               'extinout/mybackup', 1, 1);
+call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT(
+                                               'extinout/mybackup', 1);
 call SYSCS_UTIL.SYSCS_DISABLE_LOG_ARCHIVE_MODE(1);
-
+commit;
 -- make sure backup is not allowed when non-logged 
 -- operations are pending
 connect 'wombat' as c2 ;
@@ -64,12 +61,12 @@
 
 set connection c1 ;
 -- following two backup calls should fail , because they are not waiting
--- for the unlogged index creation to commit/rollback.
+-- for the unlogged index creation in anothere transaction to commit/rollback.
 
-call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE('extinout/mybackup' , 0) ;
+call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_NOWAIT('extinout/mybackup') ;
 
-call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
-                                               'extinout/mybackup', 1, 0);
+call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE_NOWAIT(
+                                               'extinout/mybackup', 1);
 
 set connection c2;
 rollback ;
@@ -97,8 +94,8 @@
 -- make sure backup does not already exists at the backup location.
 values removeDirectory('extinout/ulbackup1');
 values fileExists('extinout/ulbackup1'); 
-async bthread1 'call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE(
-                                    ''extinout/ulbackup1'' , 1)' ;
+async bthread1 'call SYSCS_UTIL.SYSCS_BACKUP_DATABASE(
+                                    ''extinout/ulbackup1'')' ;
 set connection c2;
 -- sleep for a while for the backup thread to 
 -- really get into the wait state
@@ -151,8 +148,8 @@
 values removeDirectory('extinout/ulbackup3');
 values fileExists('extinout/ulbackup3'); 
 async bthread1 
-  'call SYSCS_UTIL.SYSCS_ONLINE_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
-                                    ''extinout/ulbackup3'' , 1, 1)' ;
+  'call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE(
+                                    ''extinout/ulbackup3'' , 1)' ;
 set connection c2;
 -- sleep for a while for the backup thread to 
 -- really get into the wait state