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 2005/11/17 23:59:47 UTC

svn commit: r345355 - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/store/raw/xact/ engine/org/apache/derby/impl/store/raw/ engine/org/apache/derby/impl/store/raw/data/ engine/org/apache/derby/impl/store/raw/xact/ testing/org/apache/derbyT...

Author: mikem
Date: Thu Nov 17 14:59:38 2005
New Revision: 345355

URL: http://svn.apache.org/viewcvs?rev=345355&view=rev
Log:
DERBY-239, committing next phase checkin on behalf of Suresh Thalamati.

This patch adds code to support real-time online backup with unlogged
operations. A consistent backup can not be made if there are pending
transactions with unlogged operations or if unlogged operations 
occur when backup is in progress. Because container files can  be copied 
to the backup before the transaction is committed and the data pages are 
flushed as part of the commit. As there is no transaction log for unlogged
operations, while restoring from the backup database can not be restored to 
a consistent state. To make a consistent online backup in this scenario, this patch:

1)  blocks  online backup until all the transactions with unlogged operation are
committed/aborted. 
2)  implicitly converts all unlogged operations to logged mode for the duration
of the online backup, if they are started when backup is in progress. 

This patch also adds a test to test the online backup in parallel with some DML,

> DDL and unlogged operations. 


Added:
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest1.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackup.java
    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/OnlineBackupTest1_app.properties
Modified:
    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/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/xact/Xact.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/st_1.out
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemore.runall
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/copyfiles.ant
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/storetests/st_1.sql

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=345355&r1=345354&r2=345355&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 Nov 17 14:59:38 2005
@@ -328,6 +328,16 @@
 														 long redoLWM) 
 		throws StandardException;
 
+	
+	/*
+	 * Try setting  the transaction to be in backup blocking state.
+	 *
+	 * @return     <tt>true</tt> if the transaction can be set to a 
+	 *             blocking state. 
+	 *             <tt>false</tt> otherwise.
+	 */
+	public abstract boolean setBackupBlockingState();
+
 }
 
 

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=345355&r1=345354&r2=345355&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 Nov 17 14:59:38 2005
@@ -294,4 +294,28 @@
 	  @see org.apache.derby.iapi.store.access.AccessFactory#getTransactionInfo
 	 */
 	public TransactionInfo[] getTransactionInfo();
+
+
+	/**
+	 * Checks if there any backup blocking operations are in progress and 
+	 * stops from 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.
+	 * @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) 
+		throws StandardException ;
+
+	/**
+	 * Backup completed. Allow backup blocking operations. 
+	 */
+	public void backupFinished();
+
+													   
+
+													   
 }

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=345355&r1=345354&r2=345355&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 Nov 17 14:59:38 2005
@@ -478,11 +478,22 @@
 		if (backupDirURL != null)
 			backupDir = backupDirURL;
 
+
 		// find the user transaction, it is necessary for online backup 
 		// to open the container through page cache
-		Transaction t = findUserTransaction(ContextService.getFactory().getCurrentContextManager(), 
-											AccessFactoryGlobals.USER_TRANS_NAME);
-		backup(t, new File(backupDir));
+		Transaction t = 
+            findUserTransaction(
+                ContextService.getFactory().getCurrentContextManager(), 
+                AccessFactoryGlobals.USER_TRANS_NAME);
+
+		try {
+			canStartOnlineBackup(t, true);
+			backup(t, new File(backupDir));
+		}finally {
+			// let the xactfatory know that backup is done, so that
+			// it can allow backup blocking operations. 
+			xactFactory.backupFinished();
+		}
 	}
 
 
@@ -797,6 +808,36 @@
 		}
 	}
 
+
+	/**
+	 * Checks if the online backup can be started.
+     *
+	 * A Consistent backup can not  be made if there are any backup 
+	 * blocking operations (like unlogged operations) are in progress. 
+	 * Backup is allowed only in brand new transaction to avoid issues
+	 * like users starting a backup in the same transaction that has 
+	 * pending unlogged operations. 
+	 * 
+	 * @param wait if <tt>true</tt>, waits for  all the backup blocking 
+	 *             operation in progress to finish.
+	 * @return     <tt>true</tt> if an online backup can be made.
+	 *			   <tt>false</tt> otherwise.
+	 * @exception StandardException if the transaction that is used  
+	 *                              to start the backup is not idle.
+	 */
+	private boolean canStartOnlineBackup(Transaction t, boolean wait) 
+		throws StandardException {
+		
+		// check if the transaction is in the idle state
+		if(!t.isIdle()) {
+			// online backup can only be started in an IDLE transaction.
+			// TODO : add the exception here. 
+		}
+		
+		// check if there any backup blocking operations are in progress
+		// and stop new ones from starting until the backup is completed.
+		return xactFactory.stopBackupBlockingOperations(wait); 
+	}
 	
 	//copies the files from the backup that does not need
 	//any special handling like jars.
@@ -824,14 +865,14 @@
 			{
 				if (!privCopyDirectory(fromFile, toFile)){
 					throw StandardException.newException(
-												 SQLState.UNABLE_TO_COPY_FILE_FROM_BACKUP,
-												 fromFile, toFile);
+                         SQLState.UNABLE_TO_COPY_FILE_FROM_BACKUP, 
+                         fromFile, toFile);
 				}
 			}else{
 				if (!privCopyFile(fromFile, toFile)){
 					throw StandardException.newException(
-												 SQLState.UNABLE_TO_COPY_FILE_FROM_BACKUP,
-												 fromFile, toFile);
+                         SQLState.UNABLE_TO_COPY_FILE_FROM_BACKUP,
+                         fromFile, toFile);
 				}
 			}
 		}

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=345355&r1=345354&r2=345355&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 Nov 17 14:59:38 2005
@@ -642,13 +642,38 @@
 				mode |= 
                     (ContainerHandle.MODE_UNLOGGED | 
                      ContainerHandle.MODE_CREATE_UNLOGGED);
-            }
+            } else {
 
-			// make sure everything is logged if logArchived is turn on
-			// clear all UNLOGGED flag
-			if (!inCreateNoLog && logFactory.logArchived())
-				mode &= ~(ContainerHandle.MODE_UNLOGGED |
-						  ContainerHandle.MODE_CREATE_UNLOGGED);
+				// make sure everything is logged if logArchived is turn on
+				// clear all UNLOGGED flag
+				if (logFactory.logArchived()) {
+					mode &= ~(ContainerHandle.MODE_UNLOGGED |
+							  ContainerHandle.MODE_CREATE_UNLOGGED);
+
+				} else {
+
+					// block the online backup if the container is being 
+					// opened in unlogged mode, if the backup is already 
+					// running then convert all unlogged opens to a logged ones,
+					// otherwise onlibe backup copy will be inconsistent.
+
+					if (((mode & ContainerHandle.MODE_UNLOGGED) == 
+						 ContainerHandle.MODE_UNLOGGED) || 
+						((mode & ContainerHandle.MODE_CREATE_UNLOGGED) == 
+						 ContainerHandle.MODE_CREATE_UNLOGGED))									   
+					{
+						if(!t.setBackupBlockingState()) {
+							// when a backup is in progress transaction can not
+                            // be set to backup blocking state, so convert 
+                            // unlogged opens to logged mode.
+							mode &= ~(ContainerHandle.MODE_UNLOGGED |
+									  ContainerHandle.MODE_CREATE_UNLOGGED);
+						}
+					}
+
+				}
+
+			}
 
 			// if mode is UNLOGGED but not CREATE_UNLOGGED, then force the
 			// container from cache when the transaction commits.  For

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=345355&r1=345354&r2=345355&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 Nov 17 14:59:38 2005
@@ -242,6 +242,10 @@
     // The transaction is only allowed read operations, no log writes.
     private boolean         readOnly;
 
+	// true, if the transaction executed some operations(like unlogged
+	// operations) that block the  online backup to prevent inconsistent
+	// backup copy.
+	private boolean inBackupBlockingState;
 
 
 	/*
@@ -281,6 +285,8 @@
 
 		setIdleState();
 
+		inBackupBlockingState = false; 
+
         /*
         System.out.println("Xact.constructor: readonly = " + this.readOnly +
                 ";this = " + this);
@@ -1903,6 +1909,10 @@
 
 		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();
 		inComplete = null;
 	}
 
@@ -2308,6 +2318,27 @@
 		postCompleteMode = true;
 	}
 
+	/*
+	 * Try setting the transaction to be in backup blocking state.
+	 */
+	public boolean setBackupBlockingState() {
+		if (!inBackupBlockingState)
+			inBackupBlockingState = 
+                xactFactory.canStartBackupBlockingOperation();
+
+		return inBackupBlockingState;
+	}
+	
+	/*
+	 * Unblock the backup, if it was blocked by some operation in 
+	 * this transaction. Unbloking is done at commit/abort of this 
+	 * transaction.
+	 */
+	private void setUnblockBackupState() {
+		if (inBackupBlockingState)
+			xactFactory.backupBlockingOperationFinished();	
+		inBackupBlockingState = false;
+	}
 
 	/*
 	** 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=345355&r1=345354&r2=345355&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 Nov 17 14:59:38 2005
@@ -97,7 +97,11 @@
 	private boolean inCreateNoLog = false;	// creating database, no logging
 
 	private   XAResourceManager xa_resource;
-	
+
+	private Object   backupSemaphore = new Object();
+	private long     backupBlockingOperations = 0;
+	private boolean  inBackup = false;
+
 	/*
 	** Constructor
 	*/
@@ -961,4 +965,125 @@
 
         return(xa_resource);
     }
+
+
+	/**
+	 * Checks if a backup blocking operation can be started.
+	 *
+	 * @return     <tt>true</tt> if backup blocking operations can be started.
+	 *			   <tt>false</tt> otherwise.
+	 */
+	protected boolean canStartBackupBlockingOperation()
+	{
+		synchronized(backupSemaphore) {
+			// do not allow backup blocking operations, if online backup is
+			// is in progress.
+			if (inBackup) {
+				return false;
+			} else {
+				// not in online backup, allow backup blocking operations
+				backupBlockingOperations++;
+				return true;
+			}
+		}
+	}
+
+
+	/**
+	 * Mark that a backup blocking operation finished. 
+	 */
+	protected void backupBlockingOperationFinished()
+	{
+		synchronized(backupSemaphore) {
+			if (SanityManager.DEBUG)
+				SanityManager.ASSERT(backupBlockingOperations > 0, 
+                    "no backup blocking opeations in progress"); 
+			
+			backupBlockingOperations--;
+
+			if (inBackup) {
+				// wake up the online backupthread
+				backupSemaphore.notifyAll(); 
+			}
+		}
+	}
+
+	/**
+	 * Checks if there are any backup blocking operations in progress and 
+	 * stops 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. 
+	 * 
+	 * A Consistent backup can not be made if there are any backup 
+	 * blocking operations (like unlogged operations) are in progress
+	 *
+	 * @param wait if <tt>true</tt>, waits for the current backup blocking 
+	 *             operation in progress to finish.
+	 * @return     <tt>true</tt> if no backup blocking operations are in 
+     *             progress
+	 *             <tt>false</tt> otherwise.
+	 * @exception StandardException if interrupted or a runtime exception occurs
+	 */
+	public boolean stopBackupBlockingOperations(boolean wait) 
+		throws StandardException 
+	{
+		synchronized(backupSemaphore) {
+			if (wait) {
+				// set the inBackup state to true first to stop new backup
+				// blocking operation from starting.
+				inBackup= true;
+				try	{
+					// wait for backup blocking operation in progress to finish
+					while(backupBlockingOperations > 0)
+					{
+						try	{
+							backupSemaphore.wait();
+						}
+						catch (InterruptedException ie) {
+							// make sure we are not stuck in the backup state 
+                            // if we caught an interrupt exception and the 
+                            // calling thread may not have a chance to clear 
+                            // the in backup state.
+
+							inBackup = false;
+							backupSemaphore.notifyAll();
+							throw StandardException.interrupt(ie);
+						}
+					}
+				}
+				catch (RuntimeException rte) {
+					// make sure we are not stuck in backup state if we
+					// caught a run time exception and the calling thread may 
+                    // not have a chance to clear the in backup state.
+					inBackup= false;
+					backupSemaphore.notifyAll();
+					throw rte;		// rethrow run time exception
+				}
+			} else {
+				// check if any backup blocking operations that are in  progress
+				if (backupBlockingOperations == 0)
+					inBackup = true;
+			}
+		}
+
+		if (SanityManager.DEBUG)
+			SanityManager.ASSERT(backupBlockingOperations == 0 && 
+								 inBackup == true,
+								 "store is not in correct state for backup");
+		return inBackup;
+	}
+
+
+	/**
+	 * Backup completed. Allow backup blocking operations. 
+	 */
+	public void backupFinished()
+	{
+		synchronized(backupSemaphore) {
+			inBackup = false;
+			backupSemaphore.notifyAll();
+		}
+	}
+	
 }

Added: 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=345355&view=auto
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest1.out (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/OnlineBackupTest1.out Thu Nov 17 14:59:38 2005
@@ -0,0 +1,8 @@
+Begin Online Backup Test1
+A Transaction with Unlogged Operation Started
+BACKUP STARTED
+database shutdown properly
+Restored From the Backup
+Consistency Check is Done
+database shutdown properly
+End Online Backup Test1

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/st_1.out
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/st_1.out?rev=345355&r1=345354&r2=345355&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/st_1.out (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/st_1.out Thu Nov 17 14:59:38 2005
@@ -140,6 +140,8 @@
 0 rows inserted/updated/deleted
 ij> call SYSCS_UTIL.SYSCS_SET_DATABASE_PROPERTY('foo', 'bar');
 0 rows inserted/updated/deleted
+ij> -- backup procedures will work only in new transaction, commit the work so far.
+commit;
 ij> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE('extinout/mybackup');
 0 rows inserted/updated/deleted
 ij> call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE('extinout/mybackup3', 0);

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemore.runall
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemore.runall?rev=345355&r1=345354&r2=345355&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemore.runall (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemore.runall Thu Nov 17 14:59:38 2005
@@ -29,3 +29,4 @@
 store/holdCursorExternalSortJDBC30.sql
 store/TestDurabilityProperty.java
 store/OnlineCompressTest.java
+store/OnlineBackupTest1.java

Added: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackup.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackup.java?rev=345355&view=auto
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackup.java (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackup.java Thu Nov 17 14:59:38 2005
@@ -0,0 +1,157 @@
+/*
+
+Derby - Class org.apache.derbyTesting.functionTests.store.OnlineBackup
+
+   Copyright 2005 The Apache Software Foundation or its licensors, as applicable.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+ */
+
+package org.apache.derbyTesting.functionTests.tests.store;
+import java.sql.DriverManager;
+import java.sql.Connection;
+import java.sql.CallableStatement;
+import java.sql.SQLException;
+
+/**
+ * This class provides  functionalty for tests to perform 
+ * online backup  in a separate thread. And functions to 
+ * create/restore/rollforard recovery from the backup. 
+ *
+ * @author <a href="mailto:suresh.thalamati@gmail.com">Suresh Thalamati</a>
+ * @version 1.0
+ */
+
+public class OnlineBackup implements Runnable{
+
+	private static final String backupPath = "extinout/onlinebackuptest";
+	
+	private String dbName; // name of the database to backup
+	private boolean beginBackup = false;
+	private boolean endBackup = false;
+
+	OnlineBackup(String dbName) {
+		this.dbName = dbName;
+	}
+
+	/**
+	 * implementation of run() method in the Runnable interface, which
+	 * is invoked when a thread is started using this class object. 
+	 * 
+	 *  Performs online backup. 
+	 * 
+	 */
+	public void run()	{
+		try {
+			performBackup();
+		} catch (SQLException sqle) {
+			org.apache.derby.tools.JDBCDisplayUtil.ShowSQLException(System.out, sqle);
+			sqle.printStackTrace(System.out);
+		}
+	}
+
+	/**
+	 * Backup the database
+	 */
+	void performBackup() throws SQLException {
+		Connection conn = DriverManager.getConnection("jdbc:derby:"+ dbName);
+		CallableStatement backupStmt = 	
+			conn.prepareCall("CALL SYSCS_UTIL.SYSCS_BACKUP_DATABASE(?)");
+		backupStmt.setString(1, backupPath);
+			
+		synchronized(this)	{
+			beginBackup = true;
+			endBackup = false;
+			notifyAll();
+		}
+
+		backupStmt.execute();
+		backupStmt.close();
+		conn.close();
+
+		synchronized(this)	{
+			beginBackup = false;
+			endBackup = true;
+			notifyAll();
+		}
+	}
+
+	/**
+	 * Wait for the backup to start.
+	 */
+
+	public void waitForBackupToBegin() throws InterruptedException{
+		synchronized(this) {
+			//wait for backup to begin
+			while(!beginBackup) {
+					wait();
+			}
+		}
+	}
+	
+	/*
+	 * Wait for the backup to finish.
+	 */
+	public void waitForBackupToEnd() throws InterruptedException{
+		synchronized(this) {
+			if (!endBackup) {
+				// check if a backup has actually started by the test
+				if (!beginBackup) {
+					System.out.println("BACKUP IS NOT STARTED BY THE TEST YET");	
+				} else {
+
+					//wait for backup to finish
+					while(!endBackup) {
+						wait();
+					}
+				}
+			}
+
+		}
+	}
+
+	/**
+	 * Check if backup is running ?
+	 * @return     <tt>true</tt> if backup is running.
+	 *             <tt>false</tt> otherwise.
+	 */
+	public boolean isRunning() {
+		return beginBackup;
+	}
+	
+	/**
+	 * Create a new database from the backup copy taken earlier.
+	 * @param  newDbName   name of the database to be created.
+	 */
+	public void createFromBackup(String newDbName) throws SQLException {
+		Connection conn = 
+			DriverManager.getConnection("jdbc:derby:"+ newDbName + ";" +  
+										"createFrom=" + backupPath + "/" + 
+										dbName);
+		conn.close();
+	}
+
+	
+	/**
+	 * Restore the  database from the backup copy taken earlier.
+	 */
+	public void restoreFromBackup() throws SQLException {
+		Connection conn = 
+			DriverManager.getConnection("jdbc:derby:"+ dbName + ";" +
+										"restoreFrom=" + backupPath + "/" + 
+										dbName);
+		conn.close();
+	}
+
+}

Added: 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=345355&view=auto
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1.java (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1.java Thu Nov 17 14:59:38 2005
@@ -0,0 +1,446 @@
+/*
+
+   Derby - Class org.apache.derbyTesting.functionTests.store.OnlineBackupTest1
+
+   Copyright 2005 The Apache Software Foundation or its licensors, as applicable.
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+ */
+
+package org.apache.derbyTesting.functionTests.tests.store;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.DriverManager;
+import org.apache.derby.tools.ij;
+
+/*
+ * This class tests online backup when dml/ddl actions
+ * are running in parallel to the backup thread. 
+ *
+ * @author <a href="mailto:suresh.thalamati@gmail.com">Suresh Thalamati</a>
+ * @version 1.0
+ */
+
+public class OnlineBackupTest1 {
+
+	private static final String TEST_DATABASE_NAME = "wombat" ;
+	private static final String TEST_TABLE_NAME =    "emp";
+
+	public static void main(String[] argv) throws Throwable {
+		
+        OnlineBackupTest1 test = new OnlineBackupTest1();
+   		ij.getPropertyArg(argv); 
+
+        try {
+            test.runTest();
+        }
+        catch (SQLException sqle) {
+			dumpSQLException(sqle);
+		} 
+    }
+
+
+	/*
+	 * Test online backup with unlogged operations. And DML/DDL's
+	 * running in paralel to the backup. After the backup is complete restore
+	 * the database from the backup and performs consistency checks on the
+	 * database to make sure backup was good one.  
+	 */
+	private void runTest() throws SQLException, Exception {
+		logMessage("Begin Online Backup Test1");
+		Connection conn = ij.startJBMS();
+		conn.setAutoCommit(false);
+		DatabaseActions dbActions = new DatabaseActions(conn);
+		//create the test  table. 
+		dbActions.createTable(TEST_TABLE_NAME);
+		dbActions.startUnloggedAction(TEST_TABLE_NAME);
+		logMessage("A 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");
+		backupThread.start();	
+		// wait for the backup to start
+		backup.waitForBackupToBegin();
+		logMessage("BACKUP STARTED");
+
+		// run some dml actions in another thread
+		DatabaseActions dmlActions = new DatabaseActions(DatabaseActions.DMLACTIONS, conn);
+		Thread dmlThread = new Thread(dmlActions, "DML_THREAD");
+		dmlThread.start();
+
+		// run some DDL create/drop tables in another thread
+		DatabaseActions ddlActions = new DatabaseActions(DatabaseActions.CREATEDROPS, conn);
+		Thread ddlThread = new Thread(ddlActions, "DDL_THREAD");
+		ddlThread.start();
+
+		// 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(5000);
+			
+		// backup should not even start doing real work before the
+		// unlogged transaction is commited
+		if(!backup.isRunning())
+			logMessage("Backup is not waiting for unlogged actions to commit");
+
+		// end the unlogged work transaction.
+		dbActions.endUnloggedAction(TEST_TABLE_NAME);
+		
+		backup.waitForBackupToEnd();
+		backupThread.join();
+		dmlActions.stopActivity();
+		ddlActions.stopActivity();
+		dmlThread.join();
+		ddlThread.join();
+
+		//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");
+		runConsistencyChecker(TEST_DATABASE_NAME);
+		logMessage("Consistency Check is Done");
+		//shutdown the test db 
+		shutdown(TEST_DATABASE_NAME);
+		logMessage("End Online Backup Test1");
+	}
+
+	
+	/**
+	 * Run some consistency checks.
+	 * @param  dbName  consistency checks are performed on this database.
+	 */
+	void runConsistencyChecker(String dbName) throws SQLException {
+		Connection conn = DriverManager.getConnection("jdbc:derby:" + dbName);
+		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. 
+		conn.close();
+
+	}
+
+
+		
+	/**
+	 * Shutdown the datbase
+	 * @param  dbName  Name of the database to shutdown.
+	 */
+	void shutdown(String dbName) {
+
+		try{
+			// shutdown 
+			Connection conn = 
+				DriverManager.getConnection("jdbc:derby:" + dbName + ";shutdown=true");
+		}catch(SQLException se){
+			if (se.getSQLState() != null && se.getSQLState().equals("08006"))
+				System.out.println("database shutdown properly");
+			else
+				dumpSQLException(se);
+		}
+	}
+
+	/**
+	 * 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);
+	}
+
+	/*
+	 * This class implements some DML and DDL operations to 
+	 * run againest the datbase, when the backup is in progress. 
+	 * Some of these operations can be  run in seperate threads in a
+	 * loop until they are stopped  by some other thread. 
+	 */
+	
+	class DatabaseActions implements Runnable {
+ 
+		public static final int DMLACTIONS =   1;
+		public static final int CREATEDROPS =  2;
+
+		private static final int COMMIT =     1;
+		private static final int ROLLBACK =   2;
+		private static final int OPENTX =     3;
+
+		private int     action = 0;
+		private volatile boolean stopActivity = false ;
+		private Connection conn;
+	
+		DatabaseActions(Connection conn) {
+			this.conn = conn;
+		};
+
+		DatabaseActions(int action, Connection conn)	{
+			this.action = action;
+			this.conn = conn;
+		}
+
+		/**
+		 * stops any actions that are looping on a differt threads.
+		 */
+		public void stopActivity() {
+			stopActivity = true;
+		}
+
+		/**
+		 * implementation of run() method in the Runnable interface, which
+		 * is invoked when a thread is started using this class object. 
+		 * <p>
+		 * Performs DML ot DDL actions.
+		 */
+		 public void run() {
+			try {
+				conn.setAutoCommit(false);
+				switch(action) {
+					case DMLACTIONS :
+						performDmlActions();
+						break;
+					case CREATEDROPS:
+						performCreateDropTables() ;
+						break;
+				}
+			} catch (SQLException sqle) {
+				org.apache.derby.tools.JDBCDisplayUtil.ShowSQLException(System.out, sqle);
+				sqle.printStackTrace(System.out);
+			} 
+		}
+
+		
+		/*
+		 * Run insert, update, select on the test table in a loop.
+		 */
+		void performDmlActions() throws SQLException {
+			
+			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);
+			}
+		}
+
+
+		
+		/**
+		 * start an Unlogged operation, but don't commit the transaction.
+		 * @param  tableName  name of the table to start the unlogged operation.
+		 * @exception SQLException if any database exception occurs.
+		 */
+		void startUnloggedAction(String tableName) throws SQLException {
+			// load some data
+			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) ");
+			s.close();
+		}
+
+		
+		/**
+		 * end an Unlogged operation, commit the transaction.
+		 * @param  tableName  name of the table to end unlogged operation.
+		 * @exception SQLException if any database exception occurs.
+		 */
+		void endUnloggedAction(String tableName) throws SQLException {
+			insert(tableName, 1000, OPENTX, 10);
+			conn.commit();
+		}
+
+
+				
+		/**
+		 * Create and Drop some tables.
+		 * @exception SQLException if any database exception occurs.
+		 */
+		void performCreateDropTables() throws SQLException { 
+			
+			Statement s = conn.createStatement();
+			while(!stopActivity) {
+				for( int i = 0 ; i < 100 ; i++) {
+					String tableName = "emp" + i ;
+					createTable(tableName);
+					//load some data
+					insert(tableName, 100, COMMIT, 10);
+					if((i % 2) == 0) 
+						conn.commit();
+					else
+						conn.rollback();
+				}
+				conn.commit();
+
+				for( int i = 0 ; i < 100 ; i=i+2) {
+					String tableName = "emp" + i ;
+					s.executeUpdate("drop TABLE " + "emp" +i ); 
+					if((i % 2) == 0) 
+						conn.commit();
+					else
+						conn.rollback();
+				}
+				conn.commit();
+			}
+		}
+
+
+		
+		/**
+		 * Insert some rows into the specified table.
+		 * @param  tableName  name of the table that rows are inserted.
+		 * @param  rowCount   Number of rows to Insert.
+		 * @param  txStaus    Transacton status commit/rollback/open.
+		 * @param  commitCount After how many inserts commit/rollbacku should happen.
+		 * @exception SQLException if any database exception occurs.
+		 */
+		void insert(String tableName, int rowCount, 
+					int txStatus, int commitCount) throws SQLException {
+
+			PreparedStatement ps = conn.prepareStatement("INSERT INTO " + 
+														 tableName + 
+														 " VALUES(?,?,?)");
+			for (int i = 0; i < rowCount; i++) {
+			
+				ps.setInt(1, i); // ID
+				ps.setString(2 , "skywalker" + i);
+				ps.setFloat(3, (float)(i * 2000)); 
+				ps.executeUpdate();
+				if ((i % commitCount) == 0)
+				{
+					endTransaction(txStatus);
+				}
+			}
+
+			endTransaction(txStatus);
+			ps.close();
+		}
+
+
+
+		/**
+		 * commit/rollback the transaction. 
+		 * @param  txStaus    Transacton status commit/rollback/open.
+		 * @exception SQLException if any database exception occurs.
+		 */
+		void endTransaction(int txStatus) throws SQLException
+		{
+			switch(txStatus){
+			case COMMIT: 
+				conn.commit();
+				break;
+			case ROLLBACK:
+				conn.rollback();
+				break;
+			case OPENTX:
+				//do nothing
+				break;
+			}
+		}
+		
+		/**
+		 * update some rows in the table.
+		 * @param  tableName  name of the table that rows are inserted.
+		 * @param  rowCount   Number of rows to Insert.
+		 * @param  txStaus    Transacton status commit/rollback/open.
+		 * @param  commitCount After how many inserts commit/rollbacku should
+		 *                      happen.
+		 * @exception SQLException if any database exception occurs.
+		 */
+
+		void update(String tableName, int rowCount, 
+					int txStatus, int commitCount) throws SQLException
+		{
+
+			PreparedStatement ps = conn.prepareStatement("update " + tableName + 
+														 " SET salary=? where id=?");
+		
+			for (int i = 0; i < rowCount; i++) {
+
+				ps.setFloat(1, (float)(i * 2000 * 0.08));
+				ps.setInt(2, i); // ID
+				ps.executeUpdate();
+				if ((i % commitCount) == 0)
+				{
+					endTransaction(txStatus);
+				}
+			}
+			endTransaction(txStatus);
+			ps.close();
+		}
+
+
+		/*
+		 * read  the rows in the table. 
+		 * @param  tableName  select operation is perfomed on this table.
+		 * @exception SQLException if any database exception occurs.
+		 */
+		void select(String tableName) throws SQLException {
+		
+			Statement s = conn.createStatement();
+			ResultSet rs = s.executeQuery("SELECT ID, name from " +  
+										  tableName + " order by id" );
+			int count = 0;
+			int id = 0;
+			while(rs.next())
+			{
+				int tid = rs.getInt(1);
+				String name = rs.getString(2);
+				if(name.equals("skywalker" + id) && tid!= id)
+				{
+					logMessage("DATA IN THE TABLE IS NOT AS EXPECTED");
+					logMessage("Got :ID=" +  tid + " Name=:" + name);
+					logMessage("Expected: ID=" + id + "Name=" + "skywalker" + id );
+				}
+
+				id++;
+				count++;
+			}
+
+			rs.close();
+			s.close();
+		}
+
+		/* 
+		 * create the tables that are used by this test.
+		 * @param  tableName  Name of the table to create.
+		 * @exception SQLException if any database exception occurs.
+		 */
+		void createTable(String tableName) throws SQLException {
+
+			Statement s = conn.createStatement();
+			s.executeUpdate("CREATE TABLE " + tableName + 
+							"(id INT," +
+							"name CHAR(200),"+ 
+							"salary float)");
+			s.executeUpdate("create index " + tableName + "_id_idx on " + 
+							tableName + "(id)");
+			conn.commit();
+			s.close();
+		}
+
+	}
+}

Added: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1_app.properties
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1_app.properties?rev=345355&view=auto
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1_app.properties (added)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/OnlineBackupTest1_app.properties Thu Nov 17 14:59:38 2005
@@ -0,0 +1,21 @@
+#
+# *** DO NOT PUT PROPERTIES FOR THE DERBY SYSTEM IN THIS FILE.  THEY BELONG
+# *** IN the _derby.properties file.
+#
+# It will get handed to the test on the command line in a -p <filename>
+# argument.
+#
+# This causes ij (or the GUI on ij) to load the driver and make an
+# initial connection to the database.
+#
+#
+
+#database=jdbc:derby:wombat;create=true;logDevice=extinout/br1logDir
+usedefaults=true
+useextdirs=true
+
+#Exclude for J2ME/Foundation - test requires java.sql.DriverManager
+runwithfoundation=false
+
+#exclude with SecurityManager for now
+noSecurityManager=true

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/copyfiles.ant
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/copyfiles.ant?rev=345355&r1=345354&r2=345355&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/copyfiles.ant (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/copyfiles.ant Thu Nov 17 14:59:38 2005
@@ -141,4 +141,4 @@
 MaxLogNumber_derby.properties
 MaxLogNumberRecovery_app.properties
 MaxLogNumberRecovery_derby.properties
-
+OnlineBackupTest1_app.properties

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/storetests/st_1.sql
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/storetests/st_1.sql?rev=345355&r1=345354&r2=345355&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/storetests/st_1.sql (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/storetests/st_1.sql Thu Nov 17 14:59:38 2005
@@ -54,6 +54,8 @@
 call SYSCS_UTIL.SYSCS_UNFREEZE_DATABASE();
 call SYSCS_UTIL.SYSCS_CHECKPOINT_DATABASE();
 call SYSCS_UTIL.SYSCS_SET_DATABASE_PROPERTY('foo', 'bar');
+-- backup procedures will work only in new transaction, commit the work so far.
+commit;  
 call SYSCS_UTIL.SYSCS_BACKUP_DATABASE('extinout/mybackup');
 call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE('extinout/mybackup3', 0);
 call SYSCS_UTIL.SYSCS_BACKUP_DATABASE_AND_ENABLE_LOG_ARCHIVE_MODE('extinout/mybackup3', 1);