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 su...@apache.org on 2006/08/05 19:11:48 UTC

svn commit: r429014 [1/2] - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/services/io/ engine/org/apache/derby/iapi/store/raw/ engine/org/apache/derby/iapi/store/raw/data/ engine/org/apache/derby/iapi/store/raw/log/ engine/org/apache/derby...

Author: suresht
Date: Sat Aug  5 10:11:47 2006
New Revision: 429014

URL: http://svn.apache.org/viewvc?rev=429014&view=rev
Log:
DERBY -1156 (partial) (re) encryption of an existing database.

This patch adds code required to correctly handle crash/recovery during 
encryption of an un-encrypted database and encryption of an encrypted 
database with a new key.

Crashes before updating the database with the new encryption key 
are handled by recovery, those log records are encrypted old key. 

Crashes after updates to the service.properties with a new encryption key 
is handled by using the new status flag (derby.storage.databaseEncryptionStatus)  
to track the changes and by writing the transaction log encrypted with 
new key to a new log file. 

Logging system can handle only one encryption key, which means the 
recovery can read the log with a new key or the old key. Checkpoint 
log record is also encrypted. So it is necessary for the system
to find a checkpoint in the log that it can decrypt using the encryption 
key stored in the service.properties or the one provided by the user. 
This is ensured by  by writing  COMMIT and CHECKPOINT log records
into a new log file and delete that file on-reboot if there is a crash 
before checkpoint records are updated. 

CHECKPOINT and COMMIT is done after setting the databaseEncryptionStatus to 
IN-PROGRESS in the service.properties.On a reboot if databaseEncryptionStatus 
is  IN-PROGRESS,  then engine first checks if checkpoint is in the last 
log file , it it is then (re) encryption is complete otherwise it deletes the 
last log file before doing recovery. Because the last log file also 
had the commit record , it is also gone; Now recovery sees log only encrypted 
with the old key and no end for re-encryption transaction, so it the (re) 
encryption work rolled back and database is brought to the state it was 
before (re) encryption started. 

If engine find a checkpoint in the last log file when databaseEncryptionStatus
is IN-PROGRESS , the it is clear that checkpoint is encrypted with the new 
key; so it does any cleanup required. 


Added new test cases using debug flags to crash at critical 
point during (re) encryption and recovery of failed (re)encryption. 


Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/FileUtil.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/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/BaseDataFileFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/EncryptData.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
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/ReEncryptCrashRecovery.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/FileUtil.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/FileUtil.java?rev=429014&r1=429013&r2=429014&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/FileUtil.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/FileUtil.java Sat Aug  5 10:11:47 2006
@@ -469,6 +469,60 @@
 		return true;
 	} // end of copyFile
 
+
+    public static boolean copyFile( WritableStorageFactory storageFactory, 
+                                    StorageFile from, StorageFile to)
+    {
+        return copyFile( storageFactory, from, to, (byte[]) null);
+    }
+    
+	public static boolean copyFile( WritableStorageFactory storageFactory, 
+                                    StorageFile from, StorageFile to, 
+                                    byte[] buf)
+	{
+		InputStream from_s = null;
+		OutputStream to_s = null;
+
+		try {
+			from_s = from.getInputStream();
+			to_s = to.getOutputStream();
+
+			if (buf == null)
+				buf = new byte[BUFFER_SIZE]; // reuse this buffer to copy files
+
+			for (int bytesRead = from_s.read(buf);
+				 bytesRead != -1;
+				 bytesRead = from_s.read(buf))
+				to_s.write(buf,0,bytesRead);
+
+			from_s.close();
+			from_s = null;
+
+			storageFactory.sync( to_s, false);  // RESOLVE: sync or no sync?
+			to_s.close();
+			to_s = null;
+		}
+		catch (IOException ioe)
+		{
+			return false;
+		}
+		finally
+		{
+			if (from_s != null)
+			{
+				try { from_s.close(); }
+				catch (IOException ioe) {}
+			}
+			if (to_s != null)
+			{
+				try { to_s.close(); }
+				catch (IOException ioe) {}
+			}
+		}
+
+		return true;
+	} // end of copyFile
+
 	/**
 		Convert a file path into a File object with an absolute path
 		relative to a passed in root. If path is absolute then

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java?rev=429014&r1=429013&r2=429014&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 Sat Aug  5 10:11:47 2006
@@ -281,6 +281,52 @@
         "encryptedBootPassword";
 
 
+    /**
+     * When the datbase is getting re-encrypted old encrypted key is 
+     * stored in the service.properties until re-encyrption
+     * successfully completes or rolled back. It is really the old 
+     * encryptedkey, but the property key is called the 
+     * OldEncryptedBootPassword.
+	 */
+	public static final String OLD_ENCRYPTED_KEY = 
+        "OldEncryptedBootPassword";
+
+
+    /*
+     * Following property is used to track the status of the (re)encryption,
+     * required to bring the database back to state it was before the 
+     * (re) encryption started, id (re) encryption of the database 
+     * is aborted.
+     */
+    public static final String DB_ENCRYPTION_STATUS =
+        "derby.storage.databaseEncryptionStatus";
+
+    /* (re)encryption is in progress, if a crash 
+     *  occurs after this flag is set, 
+     * (re)encryption needs to  be undone.
+     */
+    public static final int DB_ENCRYPTION_IN_PROGRESS   = 1;
+
+    /* this flag is used to track crash during undo
+       of (re) encryption during recovery .
+    */
+    public static final int DB_ENCRYPTION_IN_UNDO      =  2;
+
+    /*
+     * Cleanup any (re) encryption related resources. 
+     */
+    public static final int  DB_ENCRYPTION_IN_CLEANUP  =  3;
+
+    
+	/**
+       A File used to save the old copy of the verify key 
+       (Attribute.CRYPTO_EXTERNAL_KEY_VERIFY_FILE) file during 
+       re-encryption of the database. 
+	 */
+	String CRYPTO_OLD_EXTERNAL_KEY_VERIFY_FILE = "verifyOldKey.dat";
+
+
+
 	/**
 	 *  for debugging, keep all transaction logs intact.
 	 */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/data/DataFactory.java?rev=429014&r1=429013&r2=429014&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 Sat Aug  5 10:11:47 2006
@@ -292,6 +292,16 @@
         throws StandardException;
 
     /*
+     * Remove old versions of the containers after (re)encryption 
+     * of the  database. 
+     * @param inRecovery  <code> true </code>, if cleanup is 
+     *                    happening during recovery.
+     * @exception StandardException Standard Derby Error Policy
+     */
+    public void removeOldVersionOfContainers(boolean inRecovery) 
+        throws StandardException;
+
+    /*
      * Set that the database is encrypted.
      */
     public void setDatabaseEncrypted();

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/log/LogFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/log/LogFactory.java?rev=429014&r1=429013&r2=429014&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 Sat Aug  5 10:11:47 2006
@@ -288,9 +288,38 @@
     public void setDatabaseEncrypted();
     
     /*
-     * setup transaction log for encryption. 
+     * set up a new log file to start writing 
+     * the log records into the new log file 
+     * after this call.
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called while re-encrypting the database 
+     * at databse boot time. 
      */
-    public  void setupLogEncryption() throws StandardException;
+    public void startNewLogFile() throws StandardException;
+
+    /*
+     * find if the checkpoint is in the last log file. 
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called only if a crash occured while 
+     * re-encrypting the database at boot time. 
+     * @return <code> true </code> if if the checkpoint is 
+     *                in the last log file, otherwise 
+     *                 <code> false </code>.
+     */
+    public boolean isCheckpointInLastLogFile() 
+        throws StandardException;
+    
+    /*
+     * delete the log file after the checkpoint. 
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called only if a crash occured while 
+     * re-encrypting the database at boot time. 
+     */
+    public void deleteLogFileAfterCheckpointLogFile() 
+        throws StandardException;
 
     
     /**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java?rev=429014&r1=429013&r2=429014&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 Sat Aug  5 10:11:47 2006
@@ -115,6 +115,8 @@
     private CipherProvider newEncryptionEngine;
 	private CipherProvider newDecryptionEngine;
 	private CipherFactory  currentCipherFactory;
+    private boolean reEncrypt = false;
+    private CipherFactory newCipherFactory = null;
 	private int counter_encrypt;
 	private int counter_decrypt;
 	private int encryptionBlockSize = RawStoreFactory.DEFAULT_ENCRYPTION_BLOCKSIZE;
@@ -127,6 +129,7 @@
     private int actionCode;
     private static final int FILE_WRITER_ACTION = 1;
     private StorageFile actionStorageFile;
+    private StorageFile actionToStorageFile;
     private boolean actionAppend;
     private static final int REGULAR_FILE_EXISTS_ACTION = 2;
     private File actionRegularFile;
@@ -148,6 +151,8 @@
     private static final int COPY_STORAGE_FILE_TO_REGULAR_ACTION = 14;
     private static final int REGULAR_FILE_GET_CANONICALPATH_ACTION = 15;
     private static final int STORAGE_FILE_GET_CANONICALPATH_ACTION = 16;
+    private static final int COPY_STORAGE_FILE_TO_STORAGE_ACTION = 17;
+    private static final int STORAGE_FILE_DELETE_ACTION = 18;
 
 	public RawStore() {
 	}
@@ -180,8 +185,6 @@
 		storageFactory = dataFactory.getStorageFactory();
 
         String restoreFromBackup = null;
-        boolean reEncrypt = false;
-        CipherFactory newCipherFactory = null;
 
 		if (properties != null)
 		{
@@ -193,176 +196,11 @@
                 restoreFromBackup =
                     properties.getProperty(Attribute.ROLL_FORWARD_RECOVERY_FROM);
 
+        }
 
-			/***********************************************
-			 * encryption
-			 **********************************************/
-            
-            // check if user has requested to encrypt the database or it is an
-            // encrypted database.
-
-            String dataEncryption = 
-                properties.getProperty(Attribute.DATA_ENCRYPTION);
-            databaseEncrypted = Boolean.valueOf(dataEncryption).booleanValue(); 
-
-
-            if (!create && restoreFromBackup == null) {
-                // check if database is already encrypted, by directly peeking at the
-                // database service propertes instead of the properties passed 
-                // to this method. By looking at properties to the boot method ,
-                // one can not differentiate if user is requesting for database
-                // encryption or the database is already encrypted because 
-                // Attribute.DATA_ENCRYPTION is used  to store in the 
-                // service properties to indicate that database
-                // is encrypted and also users can specify it as URL attribute 
-                // to encrypt and existing database. 
-                               
-                String name = Monitor.getMonitor().getServiceName(this);
-                PersistentService ps = Monitor.getMonitor().getServiceType(this);
-                String canonicalName = ps.getCanonicalServiceName(name);
-                Properties serviceprops = ps.getServiceProperties(canonicalName, 
-                                                                  (Properties)null);
-                dataEncryption = serviceprops.getProperty(Attribute.DATA_ENCRYPTION);
-                boolean encryptedDatabase = Boolean.valueOf(dataEncryption).booleanValue();
-
-                if (!encryptedDatabase  && databaseEncrypted) {
-                    // it it not an encrypted database, user is asking to 
-                    // encrypt an un-encrypted database. 
-                    encryptDatabase = true;
-                    // set database as un-encrypted, we will set it as encrypted 
-                    // after encrypting the existing data. 
-                    databaseEncrypted = false;
-                } else {
-                    // check if the user has requested to renecrypt  an
-                    // encrypted datbase with new encryption password/key.
-                    if (encryptedDatabase) {
-                        if (properties.getProperty(
-                                       Attribute.NEW_BOOT_PASSWORD) != null) {
-                            reEncrypt = true;
-                        }
-                        else if (properties.getProperty(
-                                       Attribute.NEW_CRYPTO_EXTERNAL_KEY) != null){
-                            reEncrypt = true;
-                        };
-                        encryptDatabase = reEncrypt;
-                    }
-
-                }
-                
-                // NOTE: if user specifies Attribute.DATA_ENCRYPTION on the
-                // connection URL by mistake on an already encrypted database, 
-                // it is ignored.
-
-            }
-
-            // setup encryption engines. 
-			if (databaseEncrypted || encryptDatabase)
-			{
-                // check if database is configured for encryption, during
-                // configuration  some of the properties database; so that
-                // user does not have to specify them on the URL everytime.
-                // Incase of re-encryption of an already of encrypted database
-                // only some information needs to updated; it is not treated 
-                // like the configuring the database for encryption first time. 
-                boolean setupEncryption = create || (encryptDatabase &&  !reEncrypt);
-
-                // start the cipher factory module, that is is used to create 
-                // instances of the cipher factory with specific enctyption 
-                // properties. 
-
-                CipherFactoryBuilder cb =  (CipherFactoryBuilder)
-                    Monitor.startSystemModule(org.apache.derby.iapi.reference.Module.CipherFactoryBuilder);
-
-                // create instance of the cipher factory with the 
-                // specified encryption properties. 
-                currentCipherFactory = cb.createCipherFactory(setupEncryption, 
-                                                              properties, 
-                                                              false);
-
-                // The database can be encrypted using an encryption key that is given at
-                 // connection url. For security reasons, this key is not made persistent
-                // in the database. But it is necessary to verify the encryption key 
-                // whenever booting the database if it is similar to the key that was used
-                // during creation time. This needs to happen before we access the data/logs to 
-                // avoid the risk of corrupting the database because of a wrong encryption key.
-                
-                // Please note this verification process does not provide any added security
-                // but is intended to allow to fail gracefully if a wrong encryption key 
-                // is used during boot time
-  
-
-                currentCipherFactory.verifyKey(setupEncryption, storageFactory, properties);
-
-                // Initializes the encryption and decryption engines
-                encryptionEngine = currentCipherFactory.
-                    createNewCipher(CipherFactory.ENCRYPT);
-                
-                // At creation time of an encrypted database, store the encryption block size
-                // for the algorithm. Store this value as property given by  
-                // RawStoreFactory.ENCRYPTION_BLOCKSIZE. This value
-                // is made persistent by storing it in service.properties
-                // To connect to an existing database, retrieve the value and use it for
-                // appropriate padding.
-                // The  default value of encryption block size is 8,
-                // to allow for downgrade issues
-                // Before support for AES (beetle6023), default encryption block size supported
-                // was 8
-
-                if(setupEncryption) 
-                {
-                    encryptionBlockSize = encryptionEngine.getEncryptionBlockSize();
-                    // in case of database create, store the encryption block
-                    // size. Incase of reconfiguring the existing datbase, this
-                    // will be saved after encrypting the exisiting data. 
-                    if (create)
-                        properties.put(RawStoreFactory.ENCRYPTION_BLOCKSIZE,
-                                       String.valueOf(encryptionBlockSize));
-                }
-                else
-                {
-                    if(properties.getProperty(RawStoreFactory.ENCRYPTION_BLOCKSIZE) != null)
-                        encryptionBlockSize = Integer.parseInt(properties.getProperty
-                                                               (RawStoreFactory.ENCRYPTION_BLOCKSIZE));
-                    else
-                        encryptionBlockSize = encryptionEngine.getEncryptionBlockSize();
-                }   
-
-                decryptionEngine = currentCipherFactory.
-                    createNewCipher(CipherFactory.DECRYPT);
-
-                random = currentCipherFactory.getSecureRandom();
-                    
-                if (encryptDatabase) {
-
-                    if (reEncrypt) {
-                        // create new cipher factory with the new encrytpion
-                        // properties specified by the user. This cipher factory
-                        // is used to create the new encryption/decryption
-                        // engines to reencrypt the database with the new
-                        // encryption keys. 
-                        newCipherFactory = 
-                            cb.createCipherFactory(setupEncryption, 
-                                                   properties, 
-                                                   true);
-                        newDecryptionEngine = 
-                            newCipherFactory.createNewCipher(CipherFactory.DECRYPT);
-                        newEncryptionEngine = 
-                            newCipherFactory.createNewCipher(CipherFactory.ENCRYPT);
-                    } else {
-                        // there is only one engine when configuring an 
-                        // unencrypted database for encryption 
-                        newDecryptionEngine = decryptionEngine;
-                        newEncryptionEngine = encryptionEngine;
-
-                    }
-                }
-
-                // save the encryption properties if encryption is enabled 
-                // at database creation time. 
-                if(create)
-                    currentCipherFactory.saveProperties(properties) ;
-			}
-		}
+        // setup database encryption engines.
+        if (create) 
+            setupEncryptionEngines(create, restoreFromBackup, properties);
 
 
 		// let everyone knows who their rawStoreFactory is and they can use it
@@ -382,13 +220,6 @@
 		// log factory is booted by the data factory
 		logFactory =(LogFactory) Monitor.findServiceModule(this, getLogFactoryModule());
 
-        if (databaseEncrypted) {
-            // let log factory know if the database is encrypted . 
-            logFactory.setDatabaseEncrypted();
-            // let data factory know if the database is encrypted. 
-            dataFactory.setDatabaseEncrypted();
-        }
-
 
 		//save the service properties to a file if we are doing a restore from
 		if(restoreFromBackup !=null)
@@ -453,6 +284,29 @@
 		 * it shoud boot without any problem.
 		 **/
 
+
+        // setup database encryption engine
+        if (!create) 
+        {
+            // check if the engine crashed while re-encrypting an 
+            // encrypted database or while encryption and 
+            // existing database.
+            if(properties.getProperty(
+                              RawStoreFactory.DB_ENCRYPTION_STATUS) !=null) 
+            {   
+                handleIncompleteDatabaseEncryption(properties);
+            }
+
+            setupEncryptionEngines(create, restoreFromBackup, properties);
+        }
+
+        if (databaseEncrypted) {
+            // let log factory know if the database is encrypted . 
+            logFactory.setDatabaseEncrypted();
+            // let data factory know if the database is encrypted. 
+            dataFactory.setDatabaseEncrypted();
+        }
+
 		// no need to tell log factory which raw store factory it belongs to
 		// since this is passed into the log factory for recovery
 		// after the factories are loaded, recover the database
@@ -1230,31 +1084,205 @@
 	*/
 
 
-    
+    /*
+     * Setup Encryption Engines. 
+     */
+    private void setupEncryptionEngines(boolean create, 
+                                        String restoreFromBackup, 
+                                        Properties properties) 
+        throws StandardException
+    {
+                    
+            // check if user has requested to encrypt the database or it is an
+            // encrypted database.
 
-	/**
-		Encrypt cleartext into ciphertext.
+            String dataEncryption = 
+                properties.getProperty(Attribute.DATA_ENCRYPTION);
+            databaseEncrypted = Boolean.valueOf(dataEncryption).booleanValue(); 
 
-		@see CipherProvider#encrypt
 
-		@exception StandardException Standard Cloudscape Error Policy
-	 */
-	public int encrypt(byte[] cleartext, int offset, int length,
-					   byte[] ciphertext, int outputOffset, 
-                       boolean newEngine)
-		 throws StandardException
-	{
-		if ((databaseEncrypted == false && encryptDatabase == false) || 
-            (encryptionEngine == null && newEncryptionEngine == null))
-        {
-            throw StandardException.newException(
-                        SQLState.STORE_FEATURE_NOT_IMPLEMENTED);
-        }
+            if (!create && restoreFromBackup == null) {
+                // check if database is already encrypted, by directly peeking at the
+                // database service propertes instead of the properties passed 
+                // to this method. By looking at properties to the boot method ,
+                // one can not differentiate if user is requesting for database
+                // encryption or the database is already encrypted because 
+                // Attribute.DATA_ENCRYPTION is used  to store in the 
+                // service properties to indicate that database
+                // is encrypted and also users can specify it as URL attribute 
+                // to encrypt and existing database. 
+                               
+                String name = Monitor.getMonitor().getServiceName(this);
+                PersistentService ps = Monitor.getMonitor().getServiceType(this);
+                String canonicalName = ps.getCanonicalServiceName(name);
+                Properties serviceprops = ps.getServiceProperties(canonicalName, 
+                                                                  (Properties)null);
+                dataEncryption = serviceprops.getProperty(Attribute.DATA_ENCRYPTION);
+                boolean encryptedDatabase = Boolean.valueOf(dataEncryption).booleanValue();
 
-		counter_encrypt++;
+                if (!encryptedDatabase  && databaseEncrypted) {
+                    // it it not an encrypted database, user is asking to 
+                    // encrypt an un-encrypted database. 
+                    encryptDatabase = true;
+                    // set database as un-encrypted, we will set it as encrypted 
+                    // after encrypting the existing data. 
+                    databaseEncrypted = false;
+                } else {
+                    // check if the user has requested to renecrypt  an
+                    // encrypted datbase with new encryption password/key.
+                    if (encryptedDatabase) {
+                        if (properties.getProperty(
+                                       Attribute.NEW_BOOT_PASSWORD) != null) {
+                            reEncrypt = true;
+                        }
+                        else if (properties.getProperty(
+                                       Attribute.NEW_CRYPTO_EXTERNAL_KEY) != null){
+                            reEncrypt = true;
+                        };
+                        encryptDatabase = reEncrypt;
+                    }
 
-        if (newEngine) {
-            return newEncryptionEngine.encrypt(cleartext, offset, length,
+                }
+                
+                // NOTE: if user specifies Attribute.DATA_ENCRYPTION on the
+                // connection URL by mistake on an already encrypted database, 
+                // it is ignored.
+
+            }
+
+            // setup encryption engines. 
+			if (databaseEncrypted || encryptDatabase)
+			{
+                // check if database is configured for encryption, during
+                // configuration  some of the properties database; so that
+                // user does not have to specify them on the URL everytime.
+                // Incase of re-encryption of an already of encrypted database
+                // only some information needs to updated; it is not treated 
+                // like the configuring the database for encryption first time. 
+                boolean setupEncryption = create || (encryptDatabase &&  !reEncrypt);
+
+                // start the cipher factory module, that is is used to create 
+                // instances of the cipher factory with specific enctyption 
+                // properties. 
+
+                CipherFactoryBuilder cb =  (CipherFactoryBuilder)
+                    Monitor.startSystemModule(org.apache.derby.iapi.reference.Module.CipherFactoryBuilder);
+
+                // create instance of the cipher factory with the 
+                // specified encryption properties. 
+                currentCipherFactory = cb.createCipherFactory(setupEncryption, 
+                                                              properties, 
+                                                              false);
+
+                // The database can be encrypted using an encryption key that is given at
+                 // connection url. For security reasons, this key is not made persistent
+                // in the database. But it is necessary to verify the encryption key 
+                // whenever booting the database if it is similar to the key that was used
+                // during creation time. This needs to happen before we access the data/logs to 
+                // avoid the risk of corrupting the database because of a wrong encryption key.
+                
+                // Please note this verification process does not provide any added security
+                // but is intended to allow to fail gracefully if a wrong encryption key 
+                // is used during boot time
+  
+
+                currentCipherFactory.verifyKey(setupEncryption, storageFactory, properties);
+
+                // Initializes the encryption and decryption engines
+                encryptionEngine = currentCipherFactory.
+                    createNewCipher(CipherFactory.ENCRYPT);
+                
+                // At creation time of an encrypted database, store the encryption block size
+                // for the algorithm. Store this value as property given by  
+                // RawStoreFactory.ENCRYPTION_BLOCKSIZE. This value
+                // is made persistent by storing it in service.properties
+                // To connect to an existing database, retrieve the value and use it for
+                // appropriate padding.
+                // The  default value of encryption block size is 8,
+                // to allow for downgrade issues
+                // Before support for AES (beetle6023), default encryption block size supported
+                // was 8
+
+                if(setupEncryption) 
+                {
+                    encryptionBlockSize = encryptionEngine.getEncryptionBlockSize();
+                    // in case of database create, store the encryption block
+                    // size. Incase of reconfiguring the existing datbase, this
+                    // will be saved after encrypting the exisiting data. 
+                    if (create)
+                        properties.put(RawStoreFactory.ENCRYPTION_BLOCKSIZE,
+                                       String.valueOf(encryptionBlockSize));
+                }
+                else
+                {
+                    if(properties.getProperty(RawStoreFactory.ENCRYPTION_BLOCKSIZE) != null)
+                        encryptionBlockSize = Integer.parseInt(properties.getProperty
+                                                               (RawStoreFactory.ENCRYPTION_BLOCKSIZE));
+                    else
+                        encryptionBlockSize = encryptionEngine.getEncryptionBlockSize();
+                }   
+
+                decryptionEngine = currentCipherFactory.
+                    createNewCipher(CipherFactory.DECRYPT);
+
+                random = currentCipherFactory.getSecureRandom();
+                    
+                if (encryptDatabase) {
+
+                    if (reEncrypt) {
+                        // create new cipher factory with the new encrytpion
+                        // properties specified by the user. This cipher factory
+                        // is used to create the new encryption/decryption
+                        // engines to reencrypt the database with the new
+                        // encryption keys. 
+                        newCipherFactory = 
+                            cb.createCipherFactory(setupEncryption, 
+                                                   properties, 
+                                                   true);
+                        newDecryptionEngine = 
+                            newCipherFactory.createNewCipher(CipherFactory.DECRYPT);
+                        newEncryptionEngine = 
+                            newCipherFactory.createNewCipher(CipherFactory.ENCRYPT);
+                    } else {
+                        // there is only one engine when configuring an 
+                        // unencrypted database for encryption 
+                        newDecryptionEngine = decryptionEngine;
+                        newEncryptionEngine = encryptionEngine;
+
+                    }
+                }
+
+                // save the encryption properties if encryption is enabled 
+                // at database creation time. 
+                if(create)
+                    currentCipherFactory.saveProperties(properties) ;
+			}
+    }
+    
+
+	/**
+		Encrypt cleartext into ciphertext.
+
+		@see CipherProvider#encrypt
+
+		@exception StandardException Standard Cloudscape Error Policy
+	 */
+	public int encrypt(byte[] cleartext, int offset, int length,
+					   byte[] ciphertext, int outputOffset, 
+                       boolean newEngine)
+		 throws StandardException
+	{
+		if ((databaseEncrypted == false && encryptDatabase == false) || 
+            (encryptionEngine == null && newEncryptionEngine == null))
+        {
+            throw StandardException.newException(
+                        SQLState.STORE_FEATURE_NOT_IMPLEMENTED);
+        }
+
+		counter_encrypt++;
+
+        if (newEngine) {
+            return newEncryptionEngine.encrypt(cleartext, offset, length,
                                             ciphertext, outputOffset);
         } else {
             return encryptionEngine.encrypt(cleartext, offset, length,
@@ -1324,18 +1352,56 @@
 
 
     /**
-     * Re-encryption testing debug flags, that are used to 
+     * (re) encryption testing debug flags that are used to 
      * simulate error/crash conditions for testing purposes.
-     */
-
-    /*
-     * Set to true to make the re-encryption fail just 
-     * before it is committed.
+     * When any one of the following flags are set to true
+     * in the debug mode, re-encryption will fail at that point.
      */
 
 	public static final String TEST_REENCRYPT_CRASH_BEFORE_COMMT  = 
         SanityManager.DEBUG ? "TEST_REENCRYPT_CRASH_BEFORE_COMMT" : null ;
+    public static final String TEST_REENCRYPT_CRASH_AFTER_COMMT  = 
+        SanityManager.DEBUG ? "TEST_REENCRYPT_CRASH_AFTER_COMMT" : null ;
+    public static final String TEST_REENCRYPT_CRASH_AFTER_SWITCH_TO_NEWKEY  = 
+        SanityManager.DEBUG ? "TEST_REENCRYPT_CRASH_AFTER_SWITCH_TO_NEWKEY" : null ;
+    public static final String TEST_REENCRYPT_CRASH_AFTER_CHECKPOINT  = 
+        SanityManager.DEBUG ? "TEST_REENCRYPT_CRASH_AFTER_CHECKPOINT" : null ;
+    public static final String 
+        TEST_REENCRYPT_CRASH_AFTER_RECOVERY_UNDO_LOGFILE_DELETE =
+        SanityManager.DEBUG ?
+        "TEST_REENCRYPT_CRASH_AFTER_RECOVERY_UNDO_LOGFILE_DELETE" : null;
+    public static final String 
+        TEST_REENCRYPT_CRASH_AFTER_RECOVERY_UNDO_REVERTING_KEY =
+        SanityManager.DEBUG ?
+        "TEST_REENCRYPT_CRASH_AFTER_RECOVERY_UNDO_REVERTING_KEY" : null;
+    public static final String 
+        TEST_REENCRYPT_CRASH_BEFORE_RECOVERY_FINAL_CLEANUP =
+        SanityManager.DEBUG ?
+        "TEST_REENCRYPT_CRASH_BEFORE_RECOVERY_FINAL_CLEANUP" : null;
+    
+    
 
+    /** 
+     * when the input debug flag is set, an expception 
+     * is throw when run in the debug mode.
+     */
+    private void crashOnDebugFlag(String debugFlag) 
+        throws StandardException
+    {
+        if (SanityManager.DEBUG)
+        {
+            // if the test debug flag is set, throw an 
+            // exception to simulate error cases.
+            if (SanityManager.DEBUG_ON(debugFlag))
+            {
+                StandardException se= StandardException.newException(
+                       SQLState.LOG_IO_ERROR, 
+                       new IOException(debugFlag));
+                markCorrupt(se);
+                throw se;
+            }
+        }
+    }
 
     /*
      * Configure the database for encryption, with the  specified 
@@ -1348,7 +1414,7 @@
      * new encryption  attributes at boot time only; advantage of this approach
      * is that there will not be any concurrency issues to handle because
      * no users will be modifying the data. 
-
+     *
      * First step is to encrypt the existing data with new encryption 
      * attributes  and then update the encryption properties for 
      * the database. Configuring  an un-encrypted database for 
@@ -1358,8 +1424,29 @@
      * transaction, if there is a crash/error before it is committed, 
      * then it  is rolled back and the database will be brought back to the
      * state it was before the encryption.  
+     *
+     * One trickey case in (re) encrypion of database is 
+     * unlike standard protocol transaction  commit means all done, 
+     * database (re) encryption process has to perform a checkpoint
+     *  with a newly generated key then only database  (re) encrption 
+     * is complete, Otherwise the problem  is recovery has to deal 
+     * with transaction log that is encrypted with old encryption key and 
+     * the new encryption key. This probelm is avoided  writing COMMIT
+     * and new  CHECKPOINT log record  to a new log file and encrypt the 
+     * with a new key, if there is  crash before checkpoint records 
+     * are updated , then on next boot the log file after the checkpoint 
+     * is deleted before reovery,  which will be the one that is  
+     * written with new encryption key and also contains COMMIT record, 
+     * so the COMMIT record is also gone when  log file is deleted. 
+     * Recovery will not see the commit , so it will  rollback the (re)
+     * encryption and revert all the containers to the 
+     * original versions. 
+     * 
+     * Old container versions are deleted only when the check point 
+     * with new encryption key is successful, not on post-commit. 
+     *
      * @param properties  properties related to this database.
-     * @exception StandardException Standard Cloudscape Error Policy
+     * @exception StandardException Standard Derby Error Policy
      */
     public void configureDatabaseForEncryption(Properties properties,
                                                boolean reEncrypt, 
@@ -1370,6 +1457,12 @@
         // check if the database can be encrypted.
         canEncryptDatabase(reEncrypt);
 
+        boolean externalKeyEncryption = false;
+        if (properties.getProperty(Attribute.CRYPTO_EXTERNAL_KEY) != null)
+        {
+                externalKeyEncryption = true;
+        }
+
         // check point the datase, so that encryption does not have
         // to encrypt the existing transactions logs. 
  
@@ -1387,65 +1480,169 @@
             error = false;
         }finally {
             
-            // encryption is finished. close the transaction.
+            // if (re) encryption failed, abort the transaction.
             if (error) { 
                 transaction.abort();
             }
             else {
 
-                if (SanityManager.DEBUG)
+                // (re) encryption of all the containers is complete 
+                // update the encryption properties in the 
+                // service.properties ..etc.
+
+                if (SanityManager.DEBUG) {
+                    crashOnDebugFlag(TEST_REENCRYPT_CRASH_BEFORE_COMMT);
+                }
+                
+                // let the log factory and data factory know that 
+                // database is encrypted.
+                if (!reEncrypt) {
+                    // mark in the raw store that the database is 
+                    // encrypted. 
+                    encryptDatabase = false;
+                    databaseEncrypted = true;
+                    dataFactory.setDatabaseEncrypted();
+                    logFactory.setDatabaseEncrypted();
+
+
+                } else {
+                    // switch the encryption/decryption engine to the new ones.
+                    decryptionEngine = newDecryptionEngine;  
+                    encryptionEngine = newEncryptionEngine;
+                    currentCipherFactory = newCipherFactory;
+                }
+
+  
+                // make the log factory ready to encrypt
+                // the transaction log with the new encryption 
+                // key by switching to a new log file. 
+                // If re-encryption is aborted for any reason, 
+                // this new log file will be deleted, during
+                // recovery.
+
+                logFactory.startNewLogFile();
+
+                // mark that re-encryption is in progress in the 
+                // service.properties, so that (re) encryption 
+                // changes that can not be undone using the transaction 
+                // log can be un-done before recovery starts.
+                // (like the changes to service.properties and 
+                // any log files the can not be understood by the
+                // old encryption key), incase engine crashes
+                // after this point. 
+
+                // if the crash occurs before this point, recovery
+                // will rollback the changes using the transaction 
+                // log.
+
+                properties.put(RawStoreFactory.DB_ENCRYPTION_STATUS,
+                               String.valueOf(
+                               RawStoreFactory.DB_ENCRYPTION_IN_PROGRESS));
+
+                if (reEncrypt) 
                 {
-                    // if the test debug flag is set, stop the 
-                    // re-encryption of the database here and 
-                    // throw an expception. 
-                    if (SanityManager.DEBUG_ON(TEST_REENCRYPT_CRASH_BEFORE_COMMT))
+                    // incase re-encryption, save the old 
+                    // encryption related properties, before
+                    // doing updates with new values.
+
+                    if (externalKeyEncryption) 
                     {
-                        throw StandardException.newException(
-                         SQLState.LOG_IO_ERROR, 
-                         new IOException(TEST_REENCRYPT_CRASH_BEFORE_COMMT));
+                        // save the current copy of verify key file.
+                        StorageFile verifyKeyFile = 
+                            storageFactory.newStorageFile(
+                                 Attribute.CRYPTO_EXTERNAL_KEY_VERIFY_FILE);
+                        StorageFile oldVerifyKeyFile = 
+                          storageFactory.newStorageFile(
+                          RawStoreFactory.CRYPTO_OLD_EXTERNAL_KEY_VERIFY_FILE);
+
+                        if(!privCopyFile(verifyKeyFile, oldVerifyKeyFile))
+                            throw StandardException.
+                              newException(SQLState.RAWSTORE_ERROR_COPYING_FILE,
+                                           verifyKeyFile, oldVerifyKeyFile); 
+
+                        // update the verify key file with the new key info.
+                        currentCipherFactory.verifyKey(reEncrypt, 
+                                                       storageFactory, 
+                                                       properties);
+                    } else 
+                    {
+                        // save the current generated encryption key 
+                        String keyString = 
+                            properties.getProperty(
+                                           RawStoreFactory.ENCRYPTED_KEY);
+                        if (keyString != null)
+                            properties.put(RawStoreFactory.OLD_ENCRYPTED_KEY,
+                                           keyString);
                     }
+                } else 
+                {
+                    // save the encryption block size;
+                    properties.put(RawStoreFactory.ENCRYPTION_BLOCKSIZE,
+                                   String.valueOf(encryptionBlockSize));
+                }
+
+                // save the new encryption properties into service.properties
+                currentCipherFactory.saveProperties(properties) ;
+ 
+                if (SanityManager.DEBUG) {
+                    crashOnDebugFlag(
+                                 TEST_REENCRYPT_CRASH_AFTER_SWITCH_TO_NEWKEY);
                 }
 
+                // commit the transaction that is used to 
+                // (re) encrypt the database. Note that 
+                // this will be logged with newly generated 
+                // encryption key in the new log file created 
+                // above.
                 transaction.commit();
 
-                // TODO : handle the case where if engine crashes
-                // after the commit but before the new database
-                // encryption properties are made persistent. 
-                
-                // let log factory and data factory know that 
-                // database is encrypted.
-                logFactory.setDatabaseEncrypted();
-                logFactory.setupLogEncryption();
-                dataFactory.setDatabaseEncrypted();
-                
-                // mark in the raw store that the database is 
-                // encrypted. 
-                databaseEncrypted = true;
-                encryptDatabase = false;
-                //switch the encryption/decryption engine to the new ones.
-                if (reEncrypt) {
-                    decryptionEngine = newDecryptionEngine;  
-                    encryptionEngine = newEncryptionEngine;
-                    currentCipherFactory = newCipherFactory;
+                if (SanityManager.DEBUG) {
+                    crashOnDebugFlag(TEST_REENCRYPT_CRASH_AFTER_COMMT);
                 }
 
-                //force a checkpoint with new encryption algorithm
+                // force the checkpoint with new encryption key.
                 logFactory.checkpoint(this, dataFactory, xactFactory, true);
-                // store the encryption block size;
-                properties.put(RawStoreFactory.ENCRYPTION_BLOCKSIZE,
-                               String.valueOf(encryptionBlockSize));
-                // save the encryption properties.
-                currentCipherFactory.saveProperties(properties) ;
 
-                // incase of rencrytion of database, save information needed 
-                // to verify the new key on a next boot. 
-                if (reEncrypt) {
-                    currentCipherFactory.verifyKey(reEncrypt, 
-                                               storageFactory, 
-                                               properties);
+                if (SanityManager.DEBUG) {
+                    crashOnDebugFlag(TEST_REENCRYPT_CRASH_AFTER_CHECKPOINT);
                 }
 
+                // once the checkpont makes it to the log, re-encrption 
+                // is complete. only cleanup is remaining ; update the 
+                // re-encryption status flag to cleanup. 
+                properties.put(RawStoreFactory.DB_ENCRYPTION_STATUS,
+                               String.valueOf(
+                               RawStoreFactory.DB_ENCRYPTION_IN_CLEANUP));
+
+                // database is (re)encrypted successfuly, 
+                // remove the old version of the container files.
+                dataFactory.removeOldVersionOfContainers(false);
+                
+                if (reEncrypt) 
+                {
+                    if (externalKeyEncryption)
+                    {
+                        // remove the saved copy of the verify.key file
+                        StorageFile oldVerifyKeyFile = 
+                        storageFactory.newStorageFile(
+                        RawStoreFactory.CRYPTO_OLD_EXTERNAL_KEY_VERIFY_FILE);
+                        if (!privDelete(oldVerifyKeyFile))
+                            throw StandardException.newException(
+                                    SQLState.UNABLE_TO_DELETE_FILE, 
+                                    oldVerifyKeyFile);
+                    } else 
+                    {
+                        // remove the old encryption key property.
+                        properties.remove(RawStoreFactory.OLD_ENCRYPTED_KEY);
+                    }
+                }
+
+                // (re) encrypion is done,  remove the (re) 
+                // encryption status property. 
+                properties.remove(RawStoreFactory.DB_ENCRYPTION_STATUS);
+
             }                
+
             newDecryptionEngine = null;   
             newEncryptionEngine = null;
             transaction.close(); 
@@ -1454,6 +1651,213 @@
 
 
     /**
+     * Engine might have crashed during encryption of un-encrypted datbase
+     * or while re-encryptin an already encrypted database with a new key
+     * after all the containers or (re) encrypted. If crash has occured
+     * before all containers are encrypted, recovery wil un-do re-encryption
+     * using the transaction log, nothing to be done here.
+     *
+     * If crash has occured after database encryption status flag 
+     * (RawStoreFactory.DB_ENCRYPTION_STATUS) is set, this method 
+     * will do any cleanup necessary for the recovery to correctly
+     * perform the rollback if required. 
+     *
+     *
+     *
+     * @param properties  properties related to this database.
+     * @exception StandardException Standard Derby Error Policy
+     *
+     */
+    public void handleIncompleteDatabaseEncryption(Properties properties) 
+        throws StandardException
+    {
+        // find what was the encryption status before database crashed. 
+        int dbEncryptionStatus = 0; 
+        String dbEncryptionStatusStr = 
+            properties.getProperty(RawStoreFactory.DB_ENCRYPTION_STATUS);
+        if ( dbEncryptionStatusStr != null) 
+            dbEncryptionStatus = Integer.parseInt(dbEncryptionStatusStr);
+
+        boolean reEncryption = false;
+        // check if engine crashed when (re) encryption was in progress.
+        if (dbEncryptionStatus == RawStoreFactory.DB_ENCRYPTION_IN_PROGRESS)
+        {
+
+            // check if it crashed immediately after completion or
+            // before. if the checkpoint is in the last log file 
+            // encrypted with new encryption key, it is as good 
+            // as complete. In this case just cleanup any uncleared
+            // flags and mark that database is encrypted.
+
+            if(logFactory.isCheckpointInLastLogFile()) 
+            {
+                // database (re)encryption was successful, only 
+                // cleanup is remaining. change the status to cleanup. 
+                dbEncryptionStatus = RawStoreFactory.DB_ENCRYPTION_IN_CLEANUP;
+            }else {
+
+                // crash occured before re-encrytion was completed. 
+                // update the db re-encryption status and write to 
+                // the service.properties that re-encryption 
+                // needs to be undone. The reason this status need 
+                // to be made persistent, it will help to correctly 
+                // handle a crash in this routine after the log file 
+                // encrypted with new key is deleted. If this flag
+                // is not set, on next reboot, above check 
+                // will find checkpoint in the last log file and 
+                // incorrecly assume (re) encryption is
+                // successful.
+
+                dbEncryptionStatus =  RawStoreFactory.DB_ENCRYPTION_IN_UNDO;
+                properties.put(RawStoreFactory.DB_ENCRYPTION_STATUS,
+                               String.valueOf(dbEncryptionStatus));
+            }
+        }
+
+        
+        if (dbEncryptionStatus == RawStoreFactory.DB_ENCRYPTION_IN_UNDO)
+        {
+            // delete the log file after the log file that has the checkpoint , 
+            // it has the data encrypted with the new key, including the commit
+            // record for the transaction that was used to (re)encrypt 
+            // the database. By Deleting the log file, we are forcing the
+            // recovery to rollback the (re)encryption of the database. 
+
+            logFactory.deleteLogFileAfterCheckpointLogFile();
+                
+            if (SanityManager.DEBUG) {
+                crashOnDebugFlag(
+                   TEST_REENCRYPT_CRASH_AFTER_RECOVERY_UNDO_LOGFILE_DELETE);
+            }
+
+            // Note : If a crash occurs at this point, then on reboot 
+            // it will again be in the DB_ENRYPTION_IN__UNDO state, 
+            // there will not be a file after the checkpoint log file, 
+            // so no file will be deleted. 
+
+            // check if this is a external key encryption and 
+            // if it replace the current verify key file with 
+            // the old copy. 
+
+            StorageFile verifyKeyFile = 
+                storageFactory.newStorageFile(
+                                 Attribute.CRYPTO_EXTERNAL_KEY_VERIFY_FILE);
+            
+            if (privExists(verifyKeyFile))
+            {
+                StorageFile oldVerifyKeyFile = 
+                    storageFactory.newStorageFile(
+                      RawStoreFactory.CRYPTO_OLD_EXTERNAL_KEY_VERIFY_FILE);
+            
+                if (privExists(oldVerifyKeyFile)) 
+                {
+                    if(!privCopyFile(oldVerifyKeyFile, verifyKeyFile))
+                        throw StandardException.
+                            newException(SQLState.RAWSTORE_ERROR_COPYING_FILE,
+                                         oldVerifyKeyFile, verifyKeyFile);  
+                    
+                    // only incase of re-encryption there should
+                    // be old verify key file. 
+                    reEncryption = true;
+                }else 
+                {
+                    // remove the verify key file. 
+                    if (!privDelete(verifyKeyFile))
+                        throw StandardException.newException(
+                             SQLState.UNABLE_TO_DELETE_FILE, 
+                             verifyKeyFile);
+                }
+
+            } else 
+            {
+                // database enrypted with boot password. 
+                
+                // replace the current encryption key with the old key
+                // in the service.properties file. 
+                // retreive the old encryption key 
+
+                String OldKeyString = 
+                    properties.getProperty(RawStoreFactory.OLD_ENCRYPTED_KEY);
+
+                if (OldKeyString != null) {
+                    // set the current encrypted key to the old one. 
+                    properties.put(RawStoreFactory.ENCRYPTED_KEY,
+                                   OldKeyString);
+                    
+                    // only incase of re-encryption there should
+                    // be old encryted key . 
+                    reEncryption = true;
+                }
+            }
+
+            if (!reEncryption) {
+                // crash occured when database was getting reconfigured 
+                // for encryption , all encryption properties should be 
+                // removed from service.properties
+                
+                // common props for external key or password.
+                properties.remove(Attribute.DATA_ENCRYPTION);
+                properties.remove(RawStoreFactory.LOG_ENCRYPT_ALGORITHM_VERSION);
+                properties.remove(RawStoreFactory.DATA_ENCRYPT_ALGORITHM_VERSION);
+                properties.remove(RawStoreFactory.ENCRYPTION_BLOCKSIZE);
+
+                // properties specific to password based encryption.
+                properties.remove(Attribute.CRYPTO_KEY_LENGTH);
+                properties.remove(Attribute.CRYPTO_PROVIDER);
+                properties.remove(Attribute.CRYPTO_ALGORITHM);
+                properties.remove(RawStoreFactory.ENCRYPTED_KEY);
+
+            }
+
+            if (SanityManager.DEBUG) {
+                crashOnDebugFlag(
+                    TEST_REENCRYPT_CRASH_AFTER_RECOVERY_UNDO_REVERTING_KEY);
+            }
+
+        } // end of UNDO
+
+
+        if (dbEncryptionStatus == RawStoreFactory.DB_ENCRYPTION_IN_CLEANUP)
+        {
+            // remove all the old versions of the  containers. 
+            dataFactory.removeOldVersionOfContainers(true);
+        }
+        
+        if (SanityManager.DEBUG) {
+                crashOnDebugFlag(
+                   TEST_REENCRYPT_CRASH_BEFORE_RECOVERY_FINAL_CLEANUP);
+        }
+
+        // either the (re) encryption was complete , 
+        // or undone (except for rollback that needs to be 
+        // done by the recovery). Remove re-encryption specific
+        // flags from the service.properties and old copy 
+        // of the verify key file.
+        
+        // delete the old verify key file , if it exists. 
+        StorageFile oldVerifyKeyFile = 
+            storageFactory.newStorageFile(
+                      RawStoreFactory.CRYPTO_OLD_EXTERNAL_KEY_VERIFY_FILE);
+        if (privExists(oldVerifyKeyFile)) 
+        {
+            if (!privDelete(oldVerifyKeyFile))
+                throw StandardException.newException(
+                        SQLState.UNABLE_TO_DELETE_FILE, 
+                        oldVerifyKeyFile);
+        } else 
+        {
+            // remove the old encryption key property.
+            properties.remove(RawStoreFactory.OLD_ENCRYPTED_KEY);
+        }
+
+        // remove the re-encryptin status flag. 
+        properties.remove(RawStoreFactory.DB_ENCRYPTION_STATUS);
+    }
+
+
+
+
+    /**
      * checks if the database is in the right state to (re)encrypt it.
      *
      * @param  reEncrypt true if the database getting encrypted 
@@ -1684,6 +2088,24 @@
         }
     }
 
+    private synchronized boolean privDelete(StorageFile file)
+    {
+        actionCode = STORAGE_FILE_DELETE_ACTION;
+        actionStorageFile = file;
+
+        try
+        {
+            Object ret = AccessController.doPrivileged( this);
+            return ((Boolean) ret).booleanValue();
+        }
+        catch( PrivilegedActionException pae) { return false;} // does not throw an exception
+        finally
+        {
+            actionRegularFile = null;
+        }
+    }
+
+
 
     private synchronized boolean privMkdirs( File file)
     {
@@ -1848,6 +2270,27 @@
     }
 
 
+    
+    private synchronized boolean privCopyFile( StorageFile from, StorageFile to)
+    {
+        actionCode = COPY_STORAGE_FILE_TO_STORAGE_ACTION;
+        actionStorageFile = from;
+        actionToStorageFile = to;
+
+        try
+        {
+            Object ret = AccessController.doPrivileged( this);
+            return ((Boolean) ret).booleanValue();
+        }
+        catch( PrivilegedActionException pae) { return false;} // does not throw an exception
+        finally
+        {
+            actionStorageFile = null;
+            actionToStorageFile = null;
+        }
+    }
+
+
     private synchronized String[] privList(final File file)
     {
         actionCode = REGULAR_FILE_LIST_DIRECTORY_ACTION;
@@ -1947,6 +2390,9 @@
         case REGULAR_FILE_DELETE_ACTION:
             return ReuseFactory.getBoolean(actionRegularFile.delete());
 
+        case STORAGE_FILE_DELETE_ACTION:
+            return ReuseFactory.getBoolean(actionStorageFile.delete());
+
         case REGULAR_FILE_MKDIRS_ACTION:
             // SECURITY PERMISSION - OP4
             return ReuseFactory.getBoolean(actionRegularFile.mkdirs());
@@ -2000,7 +2446,15 @@
                                            (WritableStorageFactory) storageFactory,
                                            actionStorageFile,
                                            actionRegularFile));
+
             
+        case COPY_STORAGE_FILE_TO_STORAGE_ACTION:
+            // SECURITY PERMISSION - MP1, OP4
+            return ReuseFactory.getBoolean(FileUtil.copyFile(
+                                           (WritableStorageFactory) storageFactory,
+                                           actionStorageFile,
+                                           actionToStorageFile));
+
         case REGULAR_FILE_GET_CANONICALPATH_ACTION:
             // SECURITY PERMISSION - MP1
             return (String)(actionRegularFile.getCanonicalPath());

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java?rev=429014&r1=429013&r2=429014&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 Sat Aug  5 10:11:47 2006
@@ -203,6 +203,9 @@
 
 	private Hashtable postRecoveryRemovedFiles;
 
+    private EncryptData containerEncrypter;
+
+
     // PrivilegedAction actions
     private int actionCode;
     private static final int GET_TEMP_DIRECTORY_ACTION              = 1;
@@ -2133,12 +2136,36 @@
                 ciphertext, offset, length, cleartext, outputOffset);
 	}
 
+
+    
     public void encryptAllContainers(RawTransaction t) throws StandardException
     {
-        EncryptData ed = new EncryptData(this);
+        containerEncrypter = new EncryptData(this);
         // encrypt all the conatiners in the databse
-        ed.encryptAllContainers(t);
+        containerEncrypter.encryptAllContainers(t);
     }
+
+
+    /*
+     * Remover old versions of the containers after (re)encryption 
+     * of the  database. 
+     * @param inRecovery  <code> true </code>, if cleanup is 
+     *                     happening during recovery.
+     */
+    public void removeOldVersionOfContainers(boolean inRecovery) 
+        throws StandardException
+    {
+        // check if old containers are being during recovery 
+        // because of a crash after successful completion of 
+        // (re)encryption of the  dataabase, but before the 
+        // (re)encryption cleanup  was complete. 
+        if (inRecovery) {
+            containerEncrypter = new EncryptData(this);
+        }
+        containerEncrypter.removeOldVersionOfContainers(inRecovery);
+        containerEncrypter = null;
+    }
+
 
 	/**
 		Returns the encryption block size used by the algorithm at time of

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/EncryptData.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/EncryptData.java?rev=429014&r1=429013&r2=429014&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/EncryptData.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/EncryptData.java Sat Aug  5 10:11:47 2006
@@ -62,6 +62,8 @@
 
     private BaseDataFileFactory dataFactory;
     private StorageFactory storageFactory;
+    private StorageFile[] oldFiles;
+    private int noOldFiles = 0; 
 
 	public EncryptData(BaseDataFileFactory dataFactory) {
 		this.dataFactory = dataFactory;
@@ -86,8 +88,8 @@
 
 		String[] files = dataFactory.getContainerNames();
 		if (files != null) {
-            StorageFile[] oldFiles = new StorageFile[files.length];
-            int count = 0;
+            oldFiles = new StorageFile[files.length];
+            noOldFiles = 0;
 			long segmentId = 0;
 
             // loop through all the files in seg0 and 
@@ -109,14 +111,12 @@
 
 				ContainerKey ckey = new ContainerKey(segmentId, 
                                                      containerId);
-                oldFiles[count++] = encryptContainer(t, ckey);
+                oldFiles[noOldFiles++] = encryptContainer(t, ckey);
 			}
 
-            // remove all the old versions of the 
-            // container files on post-commit.
-            Serviceable removeOldFiles = new RemoveFiles(oldFiles, count);
-            t.addPostCommitWork(removeOldFiles);
-            
+            // Old versions of the container files will
+            // be removed after the (re)encryption of database
+            // is completed. 
 		} else
 		{
 			if (SanityManager.DEBUG) 
@@ -242,6 +242,26 @@
         return sb.toString();
     }
 
+    private boolean isOldContainerFile(String fileName) 
+    {
+        // all old versions of the conatainer files
+        // start with prefix "o" and ends with ".dat"
+        if (fileName.startsWith("o") && fileName.endsWith(".dat"))
+            return true;
+        else
+            return false;
+    }
+
+    private StorageFile getFile(String ctrFileName) 
+    {
+        long segmentId = 0;
+        StringBuffer sb = new StringBuffer("seg");
+        sb.append(segmentId);
+        sb.append(storageFactory.getSeparator());
+        sb.append(ctrFileName);
+        return storageFactory.newStorageFile(sb.toString());
+    }
+
     /* Restore the contaier to the state it was before 
      * it was encrypted with new encryption key. This function is 
      * called during undo of the EncryptContainerOperation log record 
@@ -298,51 +318,57 @@
                                                  newFile);
         }
     }
-}
-
 
-/**
- * This is a helper class to remove old versions of the 
- * container files during  the post-commit of the transaction 
- * that is used to configure database with new encryption properties.
- */
-class RemoveFiles implements Serviceable 
-{
-	private StorageFile filesToGo[];
-    private int noFiles = 0 ;
-
-	RemoveFiles(StorageFile filesToGo[], int size) {
-        this.filesToGo = filesToGo;
-        this.noFiles = size;
-	}
 
-	public int performWork(ContextManager context)
-        throws StandardException  {
+    /*
+     * Remove all the old version (encrypted with old key or 
+     * un-encrypted) of the containers stored in the data directory .
+     *
+     * @param inRecovery  <code> true </code>, if cleanup is 
+     *                    happening during recovery.
+     * @exception StandardException Standard Derby Error Policy
+     */
+    public void removeOldVersionOfContainers(boolean inRecovery) 
+        throws StandardException
+    {
         
-        for (int i = 0; i < noFiles; i++) {
-            if (filesToGo[i].exists())
+        if (inRecovery) 
+        {
+            // find the old version of the container files
+            // and delete them
+            String[] files = dataFactory.getContainerNames();
+            if (files != null) 
+            {
+                // loop through all the files in seg0 and 
+                // delete all old copies of the containers.
+                for (int i = files.length-1; i >= 0 ; i--) 
+                {
+                    // if it is a old version of the container file
+                    // delete it. 
+                    if (isOldContainerFile(files[i]))
+                    {
+                        StorageFile oldFile = getFile(files[i]);
+                        if (!oldFile.delete()) 
+                        {
+                            throw StandardException.newException(
+                                          SQLState.FILE_CANNOT_REMOVE_FILE,
+                                          oldFile);
+                        }
+                    }
+                }
+            }
+        }else 
+        {
+            // delete all the old version of the containers. 
+            for (int i = 0 ; i < noOldFiles ; i++) 
             {
-                if (!filesToGo[i].delete())
+                if (!oldFiles[i].delete()) 
                 {
                     throw StandardException.newException(
-                    SQLState.FILE_CANNOT_REMOVE_FILE, filesToGo[i]);
+                                   SQLState.FILE_CANNOT_REMOVE_FILE, 
+                                   oldFiles[i]);
                 }
             }
-            
         }
-        return Serviceable.DONE;
     }
-
-
-	public boolean serviceASAP() {
-		return false;
-	}
-
-    /**
-     * delete the files immediately during the post commit.
-     * @return true, this work needs to done on user thread. 
-     */
-	public boolean serviceImmediately()	{
-		return true;
-	}	
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java?rev=429014&r1=429013&r2=429014&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 Sat Aug  5 10:11:47 2006
@@ -3980,14 +3980,83 @@
         databaseEncrypted = true;
     }
 
+
     /*
-     * setup log for encryption. 
+     * set up a new log file to start writing 
+     * the log records into the new log file 
+     * after this call.
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called while re-encrypting the database 
+     * at databse boot time. 
      */
-    public  void setupLogEncryption() throws StandardException
+    public void startNewLogFile() throws StandardException
     {
-        // switch the database to a new log file, so that 
-        // new encrytion will start on new log file. 
+        // switch the database to a new log file.
         switchLogFile();
+    }
+
+
+    /*
+     * find if the checkpoint is in the last log file. 
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called only if a crash occured while 
+     * re-encrypting the database at boot time. 
+     * @return <code> true </code> if if the checkpoint is 
+     *                in the last log file, otherwise 
+     *                 <code> false </code>.
+     */
+    public boolean isCheckpointInLastLogFile() 
+        throws StandardException
+    {
+        // check if the checkpoint is done in the last log file. 
+        long logFileNumberAfterCheckpoint = 
+            LogCounter.getLogFileNumber(checkpointInstant) + 1;
+
+        // check if there is a log file after
+        // the log file that has the last 
+        // checkpoint record.
+        StorageFile logFileAfterCheckpoint = 
+            getLogFileName(logFileNumberAfterCheckpoint);
+        // System.out.println("checking " + logFileAfterCheckpoint);
+        if (privExists(logFileAfterCheckpoint))
+            return false;
+        else 
+            return true;
+    }
+    
+    /*
+     * delete the log file after the checkpoint. 
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called only if a crash occured while 
+     * re-encrypting the database at boot time. 
+     */
+    public void deleteLogFileAfterCheckpointLogFile() 
+        throws StandardException
+    {
+        long logFileNumberAfterCheckpoint = 
+            LogCounter.getLogFileNumber(checkpointInstant) + 1;
+
+        StorageFile logFileAfterCheckpoint = 
+            getLogFileName(logFileNumberAfterCheckpoint);
+
+        // System.out.println("deleting " + logFileAfterCheckpoint);
+
+        if (privExists(logFileAfterCheckpoint)) 
+        {
+            // delete the log file (this must have beend encrypted 
+            // with the new key.
+            if (!privDelete(logFileAfterCheckpoint))
+            {
+                // throw exception, recovery can not be performed
+                // without deleting the log file encyrpted with new key.
+                throw StandardException.newException(
+                           SQLState.UNABLE_TO_DELETE_FILE, 
+                           logFileAfterCheckpoint);
+            }
+        }
     }
 
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/ReadOnly.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/ReadOnly.java?rev=429014&r1=429013&r2=429014&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 Sat Aug  5 10:11:47 2006
@@ -359,18 +359,63 @@
      * Set that the database is encrypted. Read-only database can not 
      * be reencrypted, nothing to do in this case. 
      */
-    public void setDatabaseEncrypted() {
-
+    public void setDatabaseEncrypted()
+    {
         // nothing to do for a read-only database.
     }
 
+
+    /*
+     * set up a new log file to start writing 
+     * the log records into the new log file 
+     * after this call.
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called while re-encrypting the database 
+     * at databse boot time. 
+     *
+     * Read-only database can not be reencrypted, 
+     * nothing to do in this case. 
+     */
+    public void startNewLogFile() throws StandardException 
+    {
+        // nothing to do for a read-only database. 
+    }
+
     /*
-     * setup log for encryption. Read-only database can not 
-     * be reencrypted, nothing to do in this case. 
+     * find if the checkpoint is in the last log file. 
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called only if a crash occured while 
+     * re-encrypting the database at boot time. 
+
+     * Read-only database can not be re-encrypted, 
+     * nothing to do in this case. 
      */
-    public void setupLogEncryption() throws StandardException {
+    public boolean isCheckpointInLastLogFile() 
+        throws StandardException 
+    {
         // nothing to do for a read-only database. 
+        return false;
     }
+    
+    /*
+     * delete the log file after the checkpoint. 
+     *
+     * <P>MT - synchronization provided by caller - RawStore boot,
+     * This method is called only if a crash occured while 
+     * re-encrypting the database at boot time. 
+     *
+     * Read-only database can not be re-encrypted, 
+     * nothing to do in this case. 
+     */
+    public void deleteLogFileAfterCheckpointLogFile() 
+        throws StandardException 
+    {
+        // nothing to do for a read-only database. 
+    }
+
+
 
     /**
      *  Check to see if a database has been upgraded to the required