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/10/20 07:06:46 UTC

svn commit: r326824 - in /db/derby/code/trunk/java/engine/org/apache/derby: iapi/store/raw/ iapi/store/raw/data/ iapi/store/raw/log/ impl/store/raw/ impl/store/raw/data/ impl/store/raw/log/

Author: mikem
Date: Wed Oct 19 22:06:37 2005
New Revision: 326824

URL: http://svn.apache.org/viewcvs?rev=326824&view=rev
Log:
phase 1 work for DERBY-239, committed on behalf of: Suresh Thalamati

This patch adds some code necessary to support real-time online backup that
does not block writes when database backup is in progress. All the current functional tests
passed with these changes. It would be great if some can review and commit this patch.

This patch changes the way data segment and log is backed up without blocking
the writes.

Data Segment Backup:

o The containers to be backed up are found by scanning the files in seg0.

o Each container is backed up by reading all the pages through the page cache,
and then writing to the backup container. Pages are latched while
writing into the backup container.
o Not necessary to backup containers in any particular order , All updates
that happens after a container is backed will be redone using the
transaction log on restore.

MT cases:
1) Each page is latched when it is written to the backup to prevent partial
written pages sneaking into the backup.
2) Thread that is backing up the container will stop if another thread
requests removal of the container when container is being backed up.
3) Truncate of the container blocks if the container is being backed up.
4) Partially created containers will not be backed up. Container cache will
not return the container items until the creation is complete.
(No changes are not for this case , that is how it currently works).


Transaction Log Backup:

Transaction Log file backup in two phases:
1) First Check point info and the the log files are backed up before the
data segment.
2) After the data segment is backed up , all the log files
that are generated after tha backup started are also copied into the backup.

MT cases:
1) If there is a checkpoint in progress, backup will wait for the
checkpoint to complete before copying checkpoint control information
into the backup. 

Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/log/LogFactory.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/BaseContainer.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainerHandle.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/BasePage.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/CachedPage.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/FileContainer.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InputStreamContainer.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/ReadOnly.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/ContainerHandle.java Wed Oct 19 22:06:37 2005
@@ -591,4 +591,9 @@
     **/
     public SpaceInfo getSpaceInfo() throws StandardException;
 
+	/**
+	   Backup the container to the specified path.
+	   @exception StandardException	Standard Cloudscape error policy
+	*/
+	public void backupContainer(String backupContainerPath) throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java Wed Oct 19 22:06:37 2005
@@ -33,9 +33,8 @@
 import org.apache.derby.iapi.store.raw.RecordHandle;
 import org.apache.derby.iapi.store.raw.StreamContainerHandle;
 import org.apache.derby.iapi.store.raw.xact.RawTransaction;
-
+import org.apache.derby.iapi.store.raw.Transaction;
 import org.apache.derby.io.StorageFactory;
-
 import org.apache.derby.iapi.store.access.FileResource;
 import org.apache.derby.iapi.store.access.RowSource;
 import org.apache.derby.iapi.store.raw.log.LogInstant;
@@ -43,6 +42,7 @@
 import org.apache.derby.catalog.UUID;
 
 import java.util.Properties;
+import java.io.File;
 
 public interface DataFactory extends Corruptable {
 
@@ -309,6 +309,11 @@
 	 * persistent store to proceed.
 	 */
 	public void writeFinished();
+
+	/**
+	 * Back up the data segment of the database.
+	 */
+	public void backupDataFiles(Transaction rt, File backupDir) throws StandardException;
 
     /**
      * Return an id which can be used to create a container.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/log/LogFactory.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/log/LogFactory.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/log/LogFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/log/LogFactory.java Wed Oct 19 22:06:37 2005
@@ -215,15 +215,6 @@
 
 
 	/*
-	  copies the active log files and control files to the given directory
-	  Used copy the necessary log files while doing online backup
-	  @param toDir - location where the log files should be copied to.
-	  @return true if log files copy is  successful
-	  @exception StandardException - encounter exception while doing checkpoint.
-	*/
-	public boolean copyActiveLogFiles(File toDir) throws StandardException;
-
-	/*
 	 * Enable the log archive mode, when log archive mode is 
 	 * on the system keeps all the old log files instead
 	 * of deleting them at the checkpoint.
@@ -262,6 +253,35 @@
 	 */
 	public void checkpointInRFR(LogInstant cinstant, long redoLWM, 
 								DataFactory df) throws StandardException;
+
+	
+	/*
+	 * start the transaction log backup, the transaction log is  is required
+	 * to bring the database to the consistent state on restore. 
+	 * copies the log control information , active log files to the given 
+	 * backup directory and marks that backup is in progress.
+	 @param toDir - location where the log files should be copied to.
+	 @return true if log files copy is  successful
+	 @exception StandardException - encounter exception while doing checkpoint.
+	*/
+	public void startLogBackup(File toDir) throws StandardException;
+
+	
+	/*
+	 * copy all the log files that has to go into the backup directory
+	 * and mark that backup has come to an end. 
+	 @param toDir - location where the log files should be copied to.
+	 @return true if log files copy is  successful
+	 @exception StandardException - encounter exception while doing checkpoint.
+	*/
+	public void endLogBackup(File toDir) throws StandardException;
+
+	
+	/*
+	 * Abort any activity related to backup in the log factory.
+	 * Backup is not in progress any more, it failed for some reason.
+	 **/
+	public void abortLogBackup();
 
 
 }

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=326824&r1=326823&r2=326824&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 Wed Oct 19 22:06:37 2005
@@ -23,6 +23,7 @@
 import org.apache.derby.iapi.services.daemon.DaemonFactory;
 import org.apache.derby.iapi.services.daemon.DaemonService;
 import org.apache.derby.iapi.services.context.ContextManager;
+import org.apache.derby.iapi.services.context.ContextService;
 import org.apache.derby.iapi.services.crypto.CipherFactory;
 import org.apache.derby.iapi.services.crypto.CipherProvider;
 import org.apache.derby.iapi.services.locks.LockFactory;
@@ -37,7 +38,7 @@
 
 import org.apache.derby.iapi.services.property.PersistentSet;
 import org.apache.derby.iapi.store.access.TransactionInfo;
-
+import org.apache.derby.iapi.store.access.AccessFactoryGlobals;
 import org.apache.derby.iapi.store.raw.ScanHandle;
 import org.apache.derby.iapi.store.raw.RawStoreFactory;
 import org.apache.derby.iapi.store.raw.Transaction;
@@ -97,7 +98,7 @@
 {
 	private static final String BACKUP_HISTORY = "BACKUP.HISTORY";
 	private static final String[] BACKUP_FILTER =
-	{ DataFactory.TEMP_SEGMENT_NAME, DataFactory.DB_LOCKFILE_NAME, DataFactory.DB_EX_LOCKFILE_NAME, LogFactory.LOG_DIRECTORY_NAME };
+	{ DataFactory.TEMP_SEGMENT_NAME, DataFactory.DB_LOCKFILE_NAME, DataFactory.DB_EX_LOCKFILE_NAME, LogFactory.LOG_DIRECTORY_NAME, "seg0" };
 
 	protected TransactionFactory	xactFactory;
 	protected DataFactory			dataFactory;
@@ -477,11 +478,31 @@
 		if (backupDirURL != null)
 			backupDir = backupDirURL;
 
-		backup(new File(backupDir));
+		// 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));
 	}
 
 
-	public synchronized void backup(File backupDir) throws StandardException
+	public void backup(File backupDir) throws StandardException
+	{
+		backup(backupDir);
+	}
+
+	/*
+	 * Backup the database.
+	 * Online backup copies all the database files (log, seg0  ...Etc) to the
+	 * specified backup location  without blocking any user operation for the 
+	 * whole duration of the backup. Stable copy is made using  using page level
+	 * latches 	and in some cases with the help monitors.  Transaction 
+	 * log is also backed up, this will help in bringing the databse to the
+	 * consistent state on restore.
+	 * 
+	 * TODO : make sure no parallel backup/disabling log archive mode occurs.
+	 */
+	public synchronized void backup(Transaction t, File backupDir) throws StandardException
 	{
         if (!privExists(backupDir))
 		{
@@ -502,7 +523,7 @@
                     (File) backupDir);
             }
 		}
-
+		
 		boolean error = true;
 		boolean renamed = false;
 		boolean renameFailed = false;
@@ -560,12 +581,10 @@
 				}
 			}
 
-			// checkpoint the database and freeze it
-			freeze();
 
 			// copy everything from the dataDirectory to the
-			// backup directory (except temp files)
-
+			// backup directory (except temp files, log , seg0 (see BACKUP_FILTER)
+			
             if (!privCopyDirectory(dbase, backupcopy, (byte[])null, BACKUP_FILTER))
             {
                 throw StandardException.
@@ -573,12 +592,7 @@
                                  dbase, backupcopy);
             }
 
-			logHistory(historyFile,
-                MessageService.getTextMessage(
-                    MessageId.STORE_COPIED_DB_DIR,
-                    canonicalDbName,
-                    backupcopy.getCanonicalPath()));
-
+			
 			StorageFile logdir = logFactory.getLogDirectory();
 
 			// munge service.properties file if necessary
@@ -626,7 +640,7 @@
                 privRemoveDirectory(logBackup);
 			}
 
-			//Create the log directory
+			// Create the log directory
             if (!privMkdirs(logBackup))
             {
                 throw StandardException.newException(
@@ -634,15 +648,37 @@
                     (File) logBackup);
             }
 
+			// do a checkpoint to get the persistent store up to date.
+			logFactory.checkpoint(this, dataFactory, xactFactory, true);
+			
+			// start the transaction log  backup. 
+            logFactory.startLogBackup(logBackup);
+
+			File segBackup = new File(backupcopy, "seg0");
+			
+			// Create the data segment directory
+            if (!privMkdirs(segBackup))
+            {
+                throw StandardException.newException(
+                    SQLState.RAWSTORE_CANNOT_CREATE_BACKUP_DIRECTORY,
+                    (File) segBackup);
+            }
+
+
+			// backup all the information in the data segment.
+			dataFactory.backupDataFiles(t, segBackup);
+
+			logHistory(historyFile,
+                MessageService.getTextMessage(
+                    MessageId.STORE_COPIED_DB_DIR,
+                    canonicalDbName,
+                    backupcopy.getCanonicalPath()));
+
 		
-            // copy the log to the backup location
-            if(!logFactory.copyActiveLogFiles(logBackup))
-                {
-                    throw StandardException.
-                        newException(SQLState.RAWSTORE_ERROR_COPYING_FILE,
-                                     logdir, logBackup);
-                }       
-            
+            // copy the log that got generated after the backup started to
+			// backup location and tell the logfactory that backup has come to end.
+			logFactory.endLogBackup(logBackup);
+																		  
 			logHistory(historyFile,
                 MessageService.getTextMessage(
                     MessageId.STORE_COPIED_LOG,
@@ -658,13 +694,15 @@
 		}
 		finally
 		{
-			// unfreeze db ASAP
-			unfreeze();
 
 			try
 			{
 				if (error)
 				{
+					
+					// Abort all activity related to backup in the log factory.
+					logFactory.abortLogBackup();
+
 					// remove the half backed up copy
 					// unless the error occured during  rename process;
 					// inwhich case 'backupcopy' refers to the previous backup
@@ -721,7 +759,7 @@
 											  deleteOnlineArchivedLogFiles) 
 		throws StandardException
 	{
-		enableLogArchiveMode();
+		logFactory.enableLogArchiveMode();
 		backup(backupDir);
 		//After successful backup delete the archived log files
 		//that are not necessary to do a roll-forward recovery
@@ -737,7 +775,7 @@
 											  deleteOnlineArchivedLogFiles) 
 		throws StandardException
 	{
-		enableLogArchiveMode();
+		logFactory.enableLogArchiveMode();
 		backup(backupDir);
 		//After successful backup delete the archived log files
 		//that are not necessary to do a roll-forward recovery
@@ -749,12 +787,7 @@
 	}
 
 
-	private void enableLogArchiveMode() throws StandardException
-	{
-		logFactory.enableLogArchiveMode();
-	}
-
-	public void disableLogArchiveMode(boolean deleteOnlineArchivedLogFiles)
+	public synchronized void disableLogArchiveMode(boolean deleteOnlineArchivedLogFiles)
 		throws StandardException
 	{
 		logFactory.disableLogArchiveMode();

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java Wed Oct 19 22:06:37 2005
@@ -822,6 +822,13 @@
 	 */
 	public abstract long getEstimatedPageCount(BaseContainerHandle handle, int flag) throws StandardException;
 
+	/**
+	   Backup the container to the specified path.
+	   @exception StandardException	Standard Cloudscape error policy
+	*/
+	protected abstract void  backupContainer(BaseContainerHandle handle, 
+											 String backupContainerPath) throws StandardException ;
+
 
 	/*
 	** Methods to be used by sub-classes.
@@ -854,6 +861,8 @@
 	{
 		return false;
 	}
+
+	
 
 }
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainerHandle.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainerHandle.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainerHandle.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainerHandle.java Wed Oct 19 22:06:37 2005
@@ -971,6 +971,17 @@
         return container.getSpaceInfo(this);
     }
 
+
+	/**
+	   Backup the container to the specified path.
+	   @exception StandardException	Standard Cloudscape error policy
+	*/
+	public void backupContainer(String backupContainerPath) throws StandardException 
+    {
+		checkOpen();
+		container.backupContainer(this, backupContainerPath);
+	}
+
     
     public String toString()
     {

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=326824&r1=326823&r2=326824&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 Wed Oct 19 22:06:37 2005
@@ -222,6 +222,8 @@
     private String databaseDirectory;
     private static final int RELEASE_LOCK_ON_DB_ACTION = 12;
     private static final int RESTORE_DATA_DIRECTORY_ACTION = 13;
+	private static final int GET_CONTAINER_NAMES_ACTION = 14;
+
     private String backupPath;
     private File backupRoot;
     private String[] bfilelist;
@@ -2049,6 +2051,105 @@
 		}
 	}
 
+
+	/*
+	 *  Find all the all the containers stored in the seg0 directory and 
+	 *  backup each container to the specified backup location.
+	 */
+	public void backupDataFiles(Transaction rt, File backupDir) throws StandardException
+	{
+				
+		/*
+		 * List of containers that needs to be backed up are identified by 
+		 * simply reading the list of files in seg0. 
+		 * All container that are created after the container list is created 
+		 * when backup is in progress are recreated on restore using the
+		 * transaction log.
+		 */
+
+		String[] files = getContainerNames();
+		
+		if (files != null) {
+			// No user visible locks are acquired to backup the database. A stable backup 
+			// is made by latching the pages and internal synchronization
+			// mechanisms.
+			LockingPolicy lockPolicy = 	rt.newLockingPolicy(LockingPolicy.MODE_NONE, 
+															TransactionController.ISOLATION_NOLOCK, 
+															false);
+			long segmentId = 0;
+
+			// loop through all the files in seg0 and backup all valid containers.
+			for (int f = files.length-1; f >= 0 ; f--) {
+				long containerId;
+				try	{
+					containerId = 
+						Long.parseLong(files[f].substring(1, (files[f].length() -4)), 16);
+				}
+				catch (Throwable t)
+				{
+					// ignore errors from parse, it just means that someone put
+					// a file in seg0 that we didn't expect.  Continue with the
+					// next one.
+					continue;
+				}
+
+				ContainerKey identity = new ContainerKey(segmentId, containerId);
+
+				/* Not necessary to get the container thru the transaction.
+				 * Backup opens in container in read only mode , No need to 
+				 * transition the transaction to active state. 
+				 * 
+				 *  dropped container stubs also has to be backed up 
+				 *  for restore to work correctly. That is 
+				 *  why we are using a open call that let us
+				 *  open dropped containers.
+				 */
+
+				ContainerHandle containerHdl = openDroppedContainer((RawTransaction)rt, 
+																	identity, lockPolicy, 
+																	ContainerHandle.MODE_READONLY);
+				/*
+				 * Note 1:
+				 * If a container creation is  in progress , open call will wait 
+				 * until it is complete; It will never return a handle to a 
+				 * container that is partially created. (see cache manager code
+				 * for more details)
+				 *
+				 * Note 2: 
+				 * if a container creation failed in the middle after the list 
+				 * of the names are read from seg0, it will not exist in
+				 * the database any more, so nothing to backup.  Attempt 
+				 * to open such container will return null.
+				 * 
+				 */
+
+				if( containerHdl !=  null) {
+					containerHdl.backupContainer(backupDir.getPath());
+					containerHdl.close();
+				}
+			}
+		} else
+		{
+			if (SanityManager.DEBUG) 
+				SanityManager.THROWASSERT("backup process is unable to read container names in seg0");
+		}
+	}
+
+	/**
+     * get all the names of the files in seg 0.
+	 * @return An array of all the file names in seg0.
+     **/
+	private String[] getContainerNames()
+	{
+        actionCode = GET_CONTAINER_NAMES_ACTION;
+        try{
+            return (String[]) AccessController.doPrivileged( this);
+        }
+        catch( PrivilegedActionException pae){ return null;}
+	}
+
+
+
 	/**
 	 * removes the data directory(seg*) from database home directory and
 	 * restores it from backup location.
@@ -2332,7 +2433,18 @@
         case RESTORE_DATA_DIRECTORY_ACTION:
             privRestoreDataDirectory();
             return null;
-        }
+		case GET_CONTAINER_NAMES_ACTION:
+        {
+            StorageFile seg = storageFactory.newStorageFile( "seg0");
+            if (seg.exists() && seg.isDirectory())
+            {
+                // return the  names of all files in seg0
+				return seg.list();
+            }
+            return null;
+        }  // end of case GET_CONTAINER_NAMES_ACTION
+		
+		}
         return null;
     } // end of run
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BasePage.java Wed Oct 19 22:06:37 2005
@@ -2822,6 +2822,10 @@
 	 */
 	public abstract void setContainerRowCount(long count);
 
+	/*
+	 * returns the page data array, that is actually written to the disk.
+	 */
+	protected abstract byte[] getPageArray() throws StandardException;
 												 
 	/*
 	** Debugging methods

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/CachedPage.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/CachedPage.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/CachedPage.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/CachedPage.java Wed Oct 19 22:06:37 2005
@@ -699,6 +699,19 @@
 		}
 	}
 
+
+	/*
+	 * returns the page data array, that is actually written to the disk,
+	 * when the page is cleaned from the page cache.
+	 */
+	protected byte[] getPageArray() throws StandardException 
+	{
+		writeFormatId(identity); // make subclass write the page format
+		writePage(identity);	// let subclass have a chance to write any cached
+								// data to page data array
+		return pageData;
+	}
+
 	/* methods for subclass of cached page */
 
 	// use a new pageData buffer, initialize in memory structure that depend on

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/FileContainer.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/FileContainer.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/FileContainer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/FileContainer.java Wed Oct 19 22:06:37 2005
@@ -1953,7 +1953,7 @@
 		return retval;
 	}
 
-	private long getLastPageNumber(BaseContainerHandle handle) throws StandardException
+	protected long getLastPageNumber(BaseContainerHandle handle) throws StandardException
 	{
 		long retval;
 		synchronized(allocCache)
@@ -2795,6 +2795,37 @@
 
 	}
 
+
+	/** 
+	 *  Get a latched page to write to the backup. Page Latch is necessary to 
+	 *  to prevent modification to the page when it is being backedup.
+	 *  Backup process relies on latches to get consistent snap
+	 *  shot of the page , user level table/page/row locks are NOT 
+	 *  acquired  by the online backup mechanism.
+	 *	@exception StandardException Cloudscape Standard error policy
+	 */
+	protected BasePage getPageForBackup(BaseContainerHandle handle, long pageNumber) 
+		throws StandardException 
+	{
+		PageKey pageKey = new PageKey(identity, pageNumber);
+		BasePage page = (BasePage) pageCache.find(pageKey);
+				
+		if (SanityManager.DEBUG){
+			SanityManager.ASSERT(page != null, "page is not found :" + pageKey);
+		}
+		
+        // latch the page
+        page = latchPage(handle, page, true);
+		
+		if (SanityManager.DEBUG){
+			SanityManager.ASSERT(page.isLatched(), "page is not latched:" + pageKey);
+		}
+
+		return page;
+	}
+
+	
+
 	private long getUnfilledPageNumber(BaseContainerHandle handle, long pagenum)
 		 throws StandardException
 	{
@@ -3176,5 +3207,13 @@
 		}
 		return ret;
 	}
+	
+
 
+	/**
+	   backup the container.
+	   @exception StandardException Standard Cloudscape error policy 
+	*/
+	protected abstract void backupContainer(BaseContainerHandle handle,	String backupLocation)
+	    throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InputStreamContainer.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InputStreamContainer.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InputStreamContainer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/InputStreamContainer.java Wed Oct 19 22:06:37 2005
@@ -233,4 +233,18 @@
     {
         return containerPath.getInputStream();
     }
+
+		
+	/**
+	   backup the container.
+	   @exception StandardException Standard Cloudscape error policy 
+	*/
+	protected void backupContainer(BaseContainerHandle handle,	String backupLocation)
+	    throws StandardException
+	{
+		// Nothing to do; No one would want to do a  backup of a readonly datbase. 
+		// RESOLVE: make sure some where this case is handled and 
+		// users sees a right error message.
+	}
+
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java Wed Oct 19 22:06:37 2005
@@ -44,12 +44,13 @@
 import org.apache.derby.io.WritableStorageFactory;
 import org.apache.derby.io.StorageFile;
 import org.apache.derby.io.StorageRandomAccessFile;
-
+import org.apache.derby.iapi.services.io.FileUtil;
 import java.util.Vector;
 
 import java.io.DataInput;
 import java.io.IOException;
-
+import java.io.File;
+import java.io.RandomAccessFile;
 import java.security.AccessController;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedActionException;
@@ -79,13 +80,20 @@
     private static final int REMOVE_FILE_ACTION = 3;
     private static final int OPEN_CONTAINER_ACTION = 4;
     private static final int STUBBIFY_ACTION = 5;
+	private static final int BACKUP_CONTAINER_ACTION = 6;
     private ContainerKey actionIdentity;
     private boolean actionStub;
     private boolean actionErrorOK;
     private boolean actionTryAlternatePath;
     private StorageFile actionFile;
     private LogInstant actionInstant;
-    
+	private String actionBackupLocation;
+	private BaseContainerHandle actionContainerHandle;
+
+	private boolean inBackup = false;
+	private boolean inRemove = false;
+
+
 	/*
 	 * Constructors
 	 */
@@ -152,9 +160,35 @@
 	protected void removeContainer(LogInstant instant, boolean leaveStub)
 		 throws StandardException
 	{
+
+		try {
+			synchronized(this)
+			{
+				inRemove = true;
+				// wait until the thread that is doing the backup stops 
+				// before proceeding with the remove.
+				while(inBackup)
+				{
+					try	{
+						wait();
+					}
+					catch (InterruptedException ie)
+					{
+						throw StandardException.interrupt(ie);
+					}	
+				}
+			}
+
 		// discard all of my pages in the cache
 		pageCache.discard(identity);
 		stubbify(instant);
+		}finally
+		{	
+			synchronized(this) {
+				inRemove = false;
+				notifyAll();
+			}
+		}
 
 		// RESOLVE: leaveStub false
 	}
@@ -541,6 +575,19 @@
 
         synchronized(this)
         {
+			// wait until the thread that is doing the backup completes it
+			// before truncting the container. 
+			while(inBackup)
+			{
+				try	{
+					wait();
+				}
+				catch (InterruptedException ie)
+				{
+					throw StandardException.interrupt(ie);
+				}	
+			}
+
             boolean inwrite = false;
             try
             {
@@ -838,6 +885,230 @@
         }
     }
 
+
+
+
+		
+	/**
+	   backup the  container.
+	   @exception StandardException Standard Cloudscape error policy 
+	*/
+	protected void backupContainer(BaseContainerHandle handle,	String backupLocation)
+	    throws StandardException 
+	{
+		actionContainerHandle = handle;
+        actionBackupLocation = backupLocation;
+        actionCode = BACKUP_CONTAINER_ACTION;
+        try
+        {
+            AccessController.doPrivileged(this);
+        }
+        catch( PrivilegedActionException pae){ throw (StandardException) pae.getException();}
+        finally
+        {
+            actionContainerHandle = null;
+            actionBackupLocation = null;
+        }
+	}
+
+	/**
+	 * Backup the  container.
+	 *
+	 * The container is backed up by reading all the pages through the page cache,
+	 * and then writing to the backup container if it not a committed drop
+	 * container. If the container is commited dropped one, stub is copied
+	 * to the backup using simple file copy. 
+	 *
+	 * MT scenarios:
+	 * 1) Remove and backup running in parallel thread:
+	 * The trickey case is if a request to remove the container(because of a
+	 * commited drop) comes when the conatiner backup is in progress. 
+	 * This case is handled by using the synchronization on this object monitor 
+	 * and using inRemove and inBackup flags.  Basic idea is to give 
+	 * preference to remove by stopping the backup of the container temporarily,
+	 * when  the remove container is requested by another thread. Generally,  it takes
+	 * more  time to backup a regular container than the stub becuase 
+	 * stub is just one page. After each page copy, a check is made to find 
+	 * if a remove is requested and if it is then backup of the container is
+	 * aborted and the backup thread puts itself into the wait state until
+	 * remove  request thread notifies that the remove is complete. When 
+	 * remove request compeletes stub is copies into the backup.
+	 * 
+	 * 2) Truncate and backup running in parallel:
+	 * Truncate will wait if the backup is in progress. Truncate does not
+	 * release the montitor until it is complete , backup can not start 
+	 * until it acquires, so if truncate is running, it has to release
+	 * the monitor before backup can proceed.
+	 * 
+ 	 * @exception StandardException Standard Cloudscape error policy 
+	*/
+	private void privBackupContainer(BaseContainerHandle handle,	String backupLocation)
+	    throws StandardException 
+	{
+		boolean done = true;
+		File backupFile = null;
+		RandomAccessFile backupRaf = null;
+		boolean isStub = false;
+		do {
+			try {
+
+				synchronized (this) {
+					// wait if some one is removing the container because of a drop.
+					while (inRemove)
+					{
+						try	{
+							wait();
+						}
+						catch (InterruptedException ie)
+						{
+							throw StandardException.interrupt(ie);
+						}	
+					}
+
+					if (getCommittedDropState())
+						isStub = true;
+					inBackup = true;
+				}
+			
+				// create container at the backup location.
+				if (isStub) {
+					// get the stub ( it is a committted drop table container )
+					StorageFile file = privGetFileName((ContainerKey)getIdentity(), true, false, true);
+					backupFile = new File(backupLocation, file.getName());
+
+					// directly copy the stub to the backup 
+					if(!FileUtil.copyFile(dataFactory.getStorageFactory(), file, backupFile))
+					{
+						throw StandardException.newException(SQLState.RAWSTORE_ERROR_COPYING_FILE,
+															 file, backupFile);
+					}
+				} else {
+					// regular container file 
+					StorageFile file = privGetFileName((ContainerKey)getIdentity(), false, false, true);
+					try{
+						backupFile = new File(backupLocation , file.getName());
+						backupRaf = new RandomAccessFile(backupFile,  "rw");
+					} catch (IOException ioe) {
+						throw StandardException.newException( SQLState.FILE_CREATE, ioe, backupFile);
+					}
+
+					// copy all the pages of the container from the database to the
+					// backup location by reading through the pahe cache.
+				
+					long lastPageNumber= getLastPageNumber(handle);
+					for (long pageNumber = FIRST_ALLOC_PAGE_NUMBER; 
+						 pageNumber <= lastPageNumber; pageNumber++) {
+						BasePage page = getPageForBackup(handle, pageNumber);
+						byte[] pageData = page.getPageArray();
+						writeToBackup(backupRaf, pageNumber, pageData);
+						// unlatch releases page from cache, see StoredPage.releaseExclusive()
+						page.unlatch();
+
+						// check if some one wants to commit drop the table while
+						// being backedup. If so, abort the backup and restart it 
+						// once the drop is complete.
+
+						synchronized (this)
+						{
+							if (inRemove) {
+								done = false;
+								break;
+							}
+						}
+					}
+				}	
+			} finally {
+				synchronized (this) {
+					inBackup = false;
+					notifyAll();
+				}
+			
+				// if backup of container is not complete, remove the container
+				// from the backup.
+				if (!done && backupFile != null) {
+					if (backupRaf != null) {
+						try {
+							backupRaf.close();
+							backupRaf = null;
+						} catch (IOException ioe){};
+					
+					}
+					if(backupFile.exists())
+						if (!backupFile.delete())
+							throw StandardException.newException(SQLState.UNABLE_TO_DELETE_FILE, 
+																 backupFile);
+				} else {
+					// close the backup conatiner.
+					// incase of a stub, it is already closed 
+					// while doing the copy.
+					if(!isStub) {
+						if (backupRaf != null) {
+							try {
+								backupRaf.getFD().sync();
+								backupRaf.close();
+							} catch (IOException ioe) {
+							} finally {
+								backupRaf = null;
+							}
+						}	
+					}
+				}
+			}
+	
+		} while (!done);
+	}
+
+
+	// write the page to the backup location.
+	private  void writeToBackup(RandomAccessFile backupRaf, long pageNumber, byte[] pageData) 
+		throws StandardException
+	{
+		byte[] dataToWrite;
+		
+		try {
+			if (pageNumber == FIRST_ALLOC_PAGE_NUMBER)
+			{
+				// write header into the alloc page array regardless of dirty
+				// bit because the alloc page have zero'ed out the borrowed
+				// space
+				writeHeader(pageData);
+
+				if (SanityManager.DEBUG) {
+					if (FormatIdUtil.readFormatIdInteger(pageData) != AllocPage.FORMAT_NUMBER)
+						SanityManager.THROWASSERT(
+												  "expect " +
+												  AllocPage.FORMAT_NUMBER +
+												  "got " +
+												  FormatIdUtil.readFormatIdInteger(pageData));
+				}
+
+			}
+
+			if (dataFactory.databaseEncrypted() 
+				&& pageNumber != FIRST_ALLOC_PAGE_NUMBER)
+			{
+				// We cannot encrypt the page in place because pageData is
+				// still being accessed as clear text.  The encryption
+				// buffer is shared by all who access this container and can
+				// only be used within the synchronized block.
+				dataToWrite = encryptPage(pageData, pageSize);
+			} else {
+				dataToWrite = pageData;
+			}
+			
+			long pageOffset = pageNumber * pageSize;
+			fileData.seek(pageOffset);
+			backupRaf.write(dataToWrite, 0, pageSize);
+
+		} catch (IOException ioe) {
+			// page cannot be written to the backup
+			throw StandardException.newException(
+                    SQLState.FILE_WRITE_PAGE_EXCEPTION, 
+                    ioe, getIdentity() + ":" + pageNumber);
+		}
+	}
+	
+
      // PrivilegedExceptionAction method
      public Object run() throws StandardException
      {
@@ -1112,7 +1383,14 @@
              dataFactory.stubFileToRemoveAfterCheckPoint(stub,actionInstant, getIdentity());
              return null;
          } // end of case STUBBIFY_ACTION
-         }
+		 
+		 case BACKUP_CONTAINER_ACTION: {
+			 privBackupContainer(actionContainerHandle, actionBackupLocation);
+			 return null;
+		 } // end of case BACKUP_CONTAINER_ACTION
+		 
+		 } // end of switch
          return null;
+
      } // end of run
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java Wed Oct 19 22:06:37 2005
@@ -2052,9 +2052,13 @@
 		if ((firstLogNeeded = getFirstLogNeeded(checkpoint))==-1)
 			return;
 		
+		// when  backup is in progress, logfiles should not be deleted 
+		// if they are yet to be backedup, eventhough they are not required
+		// for crash recovery.
+		firstLogNeeded = (backupInProgress ? firstLogFileToBackup : firstLogNeeded);
 		oldFirstLog = firstLogFileNumber;
 		firstLogFileNumber = firstLogNeeded;
-		
+
 		while(oldFirstLog < firstLogNeeded)
 		{
 			StorageFile uselessLogFile = null;
@@ -2100,7 +2104,7 @@
 		// one truncation at a time
 		synchronized (this)
 		{
-			firstLogNeeded = LogCounter.getLogFileNumber(checkpoint.undoLWM());
+			firstLogNeeded = (checkpoint != null ? LogCounter.getLogFileNumber(checkpoint.undoLWM()) : -1);
 
 			if (SanityManager.DEBUG)
 			{
@@ -3947,17 +3951,6 @@
 		// of a write because writing to the log file is synchronized under this.
 		synchronized(this)
 		{
-			//when the log is being  archived for roll-frward recovery
-			//we would like to switch to  a new log file.
-			//otherwise during restore  logfile in the backup could 
-			//overwrite the more uptodate log files in the 
-			//online log path. And also we would like to mark the end
-			//marker for the log file other wise during roll-forward recovery,
-			//if we see a log file with fuzzy end , we think that is the 
-			//end of the recovery.
-			if(logArchived)
-				switchLogFile();
-
 			isFrozen = true;
 		}			
 	}
@@ -4379,50 +4372,146 @@
 		deleteObsoleteLogfiles();
 	}
 
-	//copy all the active log files and the control files
-	//to the given directory from the log directory
-	public synchronized boolean copyActiveLogFiles(File toDir) throws StandardException
-	{
-		//find the first  log file number that is  active
-		long logNumber = getFirstLogNeeded(currentCheckpoint);
-		//if there is nothing to copy return
-		if (logNumber== -1)
-			return true;
 
-		StorageFile fromFile = getLogFileName(logNumber);
-		File toFile = null;
-		//copy all the active log files to the bakcup directory
-		//except the current log file , because log files is swicthed
-		//before this call when we freeze the database if the log is being 
-		//archived. If the log is not archived(the log switch does not occur in
-		//this case) copy all the log files 
-		long lastLogFileToCopy = (logArchived ? getLogFileNumber()-1 : getLogFileNumber());
-		while(logNumber <= lastLogFileToCopy)
+	private long firstLogFileToBackup ; //log file that is yet to be backedup
+	private boolean backupInProgress = false; // true if the online backup is in progress
+
+	/*
+	 * start the transaction log backup, transaction log is  is required
+	 * to bring the database to the consistent state on restore. 
+
+	 * All the log files that are created after the backup starts 
+	 * should be kept around until they are copied into the backup,
+	 * even if there are checkpoints when backup is in progress. 
+	 *
+	 * copy the control files to the backup and find first log file 
+	 * that need to be copied into the backup to bring the database
+	 * to the consistent state on restore. 
+	 * 
+	 * Log files are copied after all the data files are backed up.
+	 *
+	 */
+	public void startLogBackup(File toDir) throws StandardException
+	{
+		
+		// copy the checkpoint information into the backup, 
+		// and find the first log file that needs to be be backedup.
+		// Restore will use this checkpoint to perform recovery to bring 
+		// the database to the consistent state.
+		
+		// synchronization is necessary to make sure NO parallel 
+		// checkpoint happens when the current checkpoint information 
+		// is being copied to the backup.
+
+		synchronized(this) 
 		{
-			toFile = new File(toDir, fromFile.getName());
+			// wait until the thread that is doing the checkpoint completes it.
+			while(inCheckpoint)
+			{
+				try
+				{
+					wait();
+				}	
+				catch (InterruptedException ie)
+				{
+					throw StandardException.interrupt(ie);
+				}	
+			}
+		
+			backupInProgress = true;
+		
+			// copy the control files. 
+			StorageFile fromFile;
+			File toFile;
+			// copy the log control file
+			fromFile = getControlFileName();
+			toFile = new File(toDir,fromFile.getName());
 			if(!privCopyFile(fromFile, toFile))
-				return false;
-			fromFile = getLogFileName(++logNumber);	
-		}
+			{
+				throw StandardException.newException(SQLState.RAWSTORE_ERROR_COPYING_FILE,
+													 fromFile, toFile);
+			}
 
-		//copy the log control file
-		fromFile = getControlFileName();
-		toFile = new File(toDir,fromFile.getName());
-		if(!privCopyFile(fromFile, toFile))
-			return false;
-
-		//copy the log mirror control file
-		fromFile = getMirrorControlFileName();
-		toFile = new File(toDir,fromFile.getName());
-		if(!privCopyFile(fromFile, toFile))
-			return false;
+			// copy the log mirror control file
+			fromFile = getMirrorControlFileName();
+			toFile = new File(toDir,fromFile.getName());
+			if(!privCopyFile(fromFile, toFile))
+			{
+				throw StandardException.newException(SQLState.RAWSTORE_ERROR_COPYING_FILE,
+													 fromFile, toFile);
+			}
 
-		return true;
+			// find the first  log file number that is  active
+			firstLogFileToBackup = getFirstLogNeeded(currentCheckpoint);
+		}
+
+		// copy all the log files that has to go into the backup 
+		backupLogFiles(toDir, getLogFileNumber()-1);
 	}	
 
+	/*
+	 * copy the log files into the given backup location
+	 **/
+	private void backupLogFiles(File toDir, long lastLogFileToBackup) throws StandardException
+	{
+
+		while(firstLogFileToBackup <= lastLogFileToBackup)
+		{
+			StorageFile fromFile = getLogFileName(firstLogFileToBackup);
+			File toFile = new File(toDir, fromFile.getName());
+			if(!privCopyFile(fromFile, toFile))
+			{
+				throw StandardException.newException(SQLState.RAWSTORE_ERROR_COPYING_FILE,
+													 fromFile, toFile);
+			}
+			firstLogFileToBackup++;
+		}
+	}
+
+	/*
+	 * copy all the log files that has to go into  the backup
+	 * and mark that backup is compeleted. 
+	 */
+	public void endLogBackup(File toDir) throws StandardException
+	{
+		long lastLogFileToBackup;
+		if (logArchived)
+		{
+			// when the log is being  archived for roll-frward recovery
+			// we would like to switch to  a new log file.
+			// otherwise during restore  logfile in the backup could 
+			// overwrite the more uptodate log files in the 
+			// online log path. And also we would like to mark the end
+			// marker for the log file other wise during roll-forward recovery,
+			// if we see a log file with fuzzy end , we think that is the 
+			// end of the recovery.
+			switchLogFile();
+			lastLogFileToBackup = getLogFileNumber()-1 ;
+		}else
+		{
+			// for a plain online backup partiall filled up log file is ok, 
+			// no need to do a log switch.
+			lastLogFileToBackup = getLogFileNumber();	
+		}
+
+		// backup all the log that got generated after the backup started.
+		backupLogFiles(toDir, lastLogFileToBackup);
+
+		// mark that backup is completed.
+		backupInProgress = false;
+	}
+
+
+	/*
+	 * backup is not in progress any more, it failed for some reason.
+	 **/
+	public void abortLogBackup()
+	{
+		backupInProgress = false;
+	}
 
 
-	//Is the transaction in rollforward recovery
+	// Is the transaction in rollforward recovery
 	public boolean inRFR()
 	{
 		/*

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/ReadOnly.java
URL: http://svn.apache.org/viewcvs/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/ReadOnly.java?rev=326824&r1=326823&r2=326824&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/ReadOnly.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/ReadOnly.java Wed Oct 19 22:06:37 2005
@@ -320,9 +320,41 @@
 		//do nothing
 	}
 
-	public boolean copyActiveLogFiles(File toDir) throws StandardException
+		
+	/*
+	 * start the transaction log backup, the transaction log is  is required
+	 * to bring the database to the consistent state on restore. 
+	 * copies the log control information , active log files to the given 
+	 * backup directory and marks that backup is in progress.
+	 @param toDir - location where the log files should be copied to.
+	 @return true if log files copy is  successful
+	 @exception StandardException - encounter exception while doing checkpoint.
+	*/
+	public void startLogBackup(File toDir) throws StandardException
 	{
-		//do nothing
-		return false;
+		// nothing to do for read only databases.
+	}
+
+	
+	/*
+	 * copy all the log files that has to go into the backup directory
+	 * and mark that backup has come to an end. 
+	 @param toDir - location where the log files should be copied to.
+	 @return true if log files copy is  successful
+	 @exception StandardException - encounter exception while doing checkpoint.
+	*/
+	public void endLogBackup(File toDir) throws StandardException
+	{
+		// nothing to do for read only databases.
+	}
+
+	
+	/*
+	 * Abort any activity related to backup in the log factory.
+	 * Backup is not in progress any more, it failed for some reason.
+	 **/
+	public void abortLogBackup()
+	{
+		// nothing to do for read only databases.
 	}
 }