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/04/01 01:55:47 UTC

svn commit: r159651 [1/2] - in incubator/derby/code/trunk/java: engine/org/apache/derby/iapi/services/io/ engine/org/apache/derby/iapi/store/raw/ engine/org/apache/derby/impl/store/raw/log/ testing/org/apache/derbyTesting/functionTests/suites/ testing/org/apache/derbyTesting/unitTests/store/

Author: mikem
Date: Thu Mar 31 15:55:44 2005
New Revision: 159651

URL: http://svn.apache.org/viewcvs?view=rev&rev=159651
Log:
committing change for Suresh:  suresh.thalamati@gmail.com

Attached is a first patch towards implementing checksum support for transaction
log to handle out of order incomplete log writes during recovery.  This patch
is based on writing a checksum log record that contain checksum information for a
group of log records in the log buffers. Please refer to Derby-96 in JIRA for
more details.

Testing : Ran derbyall test suite, all tests passed.

Changes in this patch addresses writing checksum information to the transaction
log before the log records are written and verifying the log at recovery time
using the checksum information on the disk.

Writing Checksum Log Records:
Checksum log record contains checksum Algorithm, size of  the data  and the checksum value.
Added a new class to implement this log operation.

The checksum Log record is placed before the actual log data the checksum
record represents. This is done by reserving the space in the log buffers and
in the log file then writing into reserved buffer space the checksum log record whenever
buffer is full or it need to be written because of a flush request due to a
commit. Incase of a large log records that does not fit into a single log
buffer, the log records are written directly to the log file, in this case
checksum log record represents only one log record and it is written to the
log file before writing the large log record directly into the log file.

In the current system the log group information is encrypted when a database
is encrypted. There is no facility to identify that a log record is checksum
log record without decrypting the log record. Checksum Log Record is also
encrypted to work correctly with the rest of the system.

changed files: LogAccessFile.java, ChecksumOperation.java, LogToFile.java

Verifying the Log:
During recovery, while doing forward scan whenever scan finds a checksum record,
it reads the amount of the data specified in the checksum record , calculates
the checksum for the data and compares it to the checksum value in the log record. If the on
disk checksum value does not match with the value recalculated then that
portion of the log is assumed as incompletely written and the record before the
checksum log record becomes the last valid record for the recovery.

changed files: Scan.java

Unit Tests:
Enhanced a existing  recovery unit test with test cases that simulate out of
order incomplete writes by  corrupting the end of the log intentionally.

changed files: T_RecoveryBadLog.java

functional tests: current backup and recovery tests checks all cases where
checksum should be valid.

To Be Done:
0) Handle Soft upgrade from old versions.   1) Check  for any timing issues between Checkpoint Thread and Checksum Log Record writes.
2) Check  for any timing issues between Checksum Log Record writes and the backup/restore.
3) Performance Testing
4) Write a  JDBC level functional test with large log records(>32k> incomplete log writes.
5) Any thing else that I find  while testing :-)


Thanks
-suresht 


Modified:
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Loggable.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFile.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFileBuffer.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogRecord.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/Scan.java
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storeunit.runall
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_RecoverBadLog.java

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java Thu Mar 31 15:55:44 2005
@@ -512,8 +512,8 @@
         /* 450 */       "org.apache.derby.iapi.types.DTSClassInfo", //InstanceGetter,
 
  
-        /* 451 */   "org.apache.derby.catalog.types.RoutineAliasInfo"
-
-
+        /* 451 */   "org.apache.derby.catalog.types.RoutineAliasInfo",
+		/* 452 */   null,
+		/* 453 */   "org.apache.derby.impl.store.raw.log.ChecksumOperation"
 };
 }

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java Thu Mar 31 15:55:44 2005
@@ -1682,6 +1682,10 @@
         public static final int LOGOP_REMOVE_FILE =
                 (MIN_ID_2 + 291);
 
+        /* org.apache.derby.impl.store.raw.log.ChecksumOperation */
+        public static final int LOGOP_CHECKSUM =
+                (MIN_ID_2 + 453);
+
 
         /*******************************************************************
         **
@@ -1802,7 +1806,7 @@
          * Make sure this is updated when a new module is added
          */
         public static final int MAX_ID_2 =
-                (MIN_ID_2 + 452);
+                (MIN_ID_2 + 453);
 
         // DO NOT USE 4 BYTE IDS ANYMORE
         static public final int MAX_ID_4 =

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Loggable.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Loggable.java?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Loggable.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Loggable.java Thu Mar 31 15:55:44 2005
@@ -160,6 +160,8 @@
 
 	public static final int RAWSTORE =		  0x100;	// a log record generated by the raw store
 	public static final int FILE_RESOURCE =   0x400;    // related to "non-transactional" files.
+	public static final int CHECKSUM =        0x800;    // a checksum log record 
+
 
 	/**
 		Get the loggable's group value

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFile.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFile.java?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFile.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFile.java Thu Mar 31 15:55:44 2005
@@ -33,6 +33,9 @@
 import java.io.InterruptedIOException;
 import java.util.LinkedList;
 
+import org.apache.derby.iapi.services.io.FormatIdOutputStream;
+import org.apache.derby.iapi.services.io.ArrayOutputStream;
+
 
 /**
 	Wraps a RandomAccessFile file to provide buffering
@@ -56,8 +59,27 @@
 	freeBuffers --> dirtyBuffers --> freeBuffers. Movement of buffers from one
     stage to 	another stage is synchronized using	the object(this) of this class. 
 
+	A Checksum log record that has the checksum value for the data that is
+    being written to the disk is generated and written 	before the actual data. 
+	Except for the large log records that does not fit into a single buffer, 
+    checksum is calcualted for a group of log records that are in the buffer 
+	when buffers is switched. Checksum log record is written into the reserved
+	space in the beginning buffer. 
+
+    In case of a large log record that does not fit into a bufffer, it needs to 
+    be written directly to the disk instead of going through the log buffers. 
+    In this case the log record write gets broken into three parts:
+        1) Write checksum log record and LOG RECORD HEADER (length + instant) 
+        2) Write the log record. 
+        3) Write the trailing length of the log record. 
+
+	Checksum log records helps in identifying the incomplete log disk writes during 
+    recovery. This is done by recalculating the checksum value for the data on
+    the disk and comparing it to the the value stored in the checksum log
+    record. 
+
 */
-public class LogAccessFile extends OutputStream 
+public class LogAccessFile 
 {
 
     /**
@@ -67,7 +89,8 @@
      *     int   trailing length    : 4 bytes
      **/
     private static final int            LOG_RECORD_FIXED_OVERHEAD_SIZE = 16;
-
+	private static final int            LOG_RECORD_HEADER_SIZE = 12; //(length + instant)
+	private static final int            LOG_RECORD_TRAILER_SIZE = 4; //trailing length 
     private static final int            LOG_NUMBER_LOG_BUFFERS = 3;
 
 
@@ -75,7 +98,7 @@
 	private LinkedList    dirtyBuffers; //list of dirty buffers to flush
 	private  LogAccessFileBuffer currentBuffer; //current active buffer
 	private boolean flushInProgress = false;
-
+	
 	private final StorageRandomAccessFile  log;
 
 	// log can be touched only inside synchronized block protected by
@@ -85,9 +108,23 @@
 	static int                      mon_numWritesToLog;
 	static int                      mon_numBytesToLog;
 
-	public LogAccessFile(
-    StorageRandomAccessFile    log, 
-    int                 bufferSize) throws IOException 
+
+	//streams used to generated check sume log record ; see if there is any simpler way
+	private ArrayOutputStream logOutputBuffer;
+	private FormatIdOutputStream logicalOut;
+	private boolean directWrite = false; //true when log is written directly to file.
+	private long checksumInstant = -1;
+	private int checksumLength;
+	private int checksumLogRecordSize;      //checksumLength + LOG_RECORD_FIXED_OVERHEAD_SIZE
+	private boolean writeChecksum = true;  //gets set to false incase of a soft upgrade.
+	private ChecksumOperation checksumLogOperation;
+	private LogRecord checksumLogRecord;
+	private LogToFile logFactory;
+	private boolean databaseEncrypted=false;
+		
+	public LogAccessFile(LogToFile logFactory,
+						 StorageRandomAccessFile    log, 
+						 int                 bufferSize) 
     {
 		if (SanityManager.DEBUG)
 		{
@@ -97,6 +134,7 @@
 		
 		this.log            = log;
 		logFileSemaphore    = log;
+		this.logFactory     = logFactory;
 
 		if (SanityManager.DEBUG)
             SanityManager.ASSERT(LOG_NUMBER_LOG_BUFFERS >= 1);
@@ -115,9 +153,56 @@
 
 		currentBuffer = (LogAccessFileBuffer) freeBuffers.removeFirst();
 
+		if(writeChecksum)
+		{
+			/**
+			 * setup structures that are required to write the checksum log records
+			 * for a group of log records are being written to the disk. 
+			 */
+			checksumLogOperation = new ChecksumOperation();
+			checksumLogOperation.init();
+			checksumLogRecord = new LogRecord();
+
+			// Note: Checksum log records are not related any particular transaction, 
+			// they are written to store a checksum information identify
+			// incomplete log record writes. No transacton id is set for this
+			// log record. That is why a null argument is passed below 
+			// setValue(..) call. 
+			checksumLogRecord.setValue(null, checksumLogOperation);
+
+			checksumLength = 
+				checksumLogRecord.getStoredSize(checksumLogOperation.group(), null) + 
+				checksumLogOperation.getStoredSize();
+
+			// calculate checksum log operation length when the database is encrypted
+			if (logFactory.databaseEncrypted())
+			{
+				checksumLength =  logFactory.getEncryptedDataLength(checksumLength);
+				databaseEncrypted = true;
+			}
+			checksumLogRecordSize = checksumLength  + LOG_RECORD_FIXED_OVERHEAD_SIZE;
+
+			//streams required to convert a log record to raw byte array. 
+			logOutputBuffer = new ArrayOutputStream(); 
+			logicalOut = new FormatIdOutputStream(logOutputBuffer);
+
+			/** initialize the buffer with space reserved for checksum log record in
+			 * the beginning of the log buffer; checksum record is written into
+			 * this space when buffer is switched or while doing direct write to the log file.
+			 */
+		}else
+		{
+			//checksumming of transaction log feature is not in use. 
+			checksumLogRecordSize = 0;
+		}
+		
+		currentBuffer.init(checksumLogRecordSize);
 	}
 
 
+	private byte[] db = new byte[LOG_RECORD_TRAILER_SIZE]; 
+
+
     /**
      * Write a single log record to the stream.
      * <p>
@@ -164,34 +249,16 @@
     {
         int total_log_record_length = length + LOG_RECORD_FIXED_OVERHEAD_SIZE;
 
-        if (total_log_record_length > currentBuffer.bytes_free && 
-            total_log_record_length <= currentBuffer.buffer.length) 
-        {
-            // If the whole record will fit in an empty buffer, flush this
-            // one now and put this record into the next one.
-            switchLogBuffer();
-        }
-
 		if (total_log_record_length <= currentBuffer.bytes_free)
         {
             byte[] b    = currentBuffer.buffer;
             int    p    = currentBuffer.position;
 
             // writeInt(length)
-            b[p++] = (byte) ((length >>> 24) & 0xff); 
-            b[p++] = (byte) ((length >>> 16) & 0xff); 
-            b[p++] = (byte) ((length >>>  8) & 0xff); 
-            b[p++] = (byte) ((length       ) & 0xff);
+			p = writeInt(length, b, p);
             
             // writeLong(instant)
-            b[p++] = (byte) (((int)(instant >>> 56)) & 0xff); 
-            b[p++] = (byte) (((int)(instant >>> 48)) & 0xff); 
-            b[p++] = (byte) (((int)(instant >>> 40)) & 0xff); 
-            b[p++] = (byte) (((int)(instant >>> 32)) & 0xff); 
-            b[p++] = (byte) (((int)(instant >>> 24)) & 0xff); 
-            b[p++] = (byte) (((int)(instant >>> 16)) & 0xff); 
-            b[p++] = (byte) (((int)(instant >>>  8)) & 0xff); 
-            b[p++] = (byte) (((int)(instant       )) & 0xff); 
+			p = writeLong(instant, b , p);
 
             // write(data, data_offset, length - optional_data_length)
             int transfer_length = (length - optional_data_length);
@@ -213,55 +280,101 @@
             }
 
             // writeInt(length)
-            b[p++] = (byte) ((length >>> 24) & 0xff); 
-            b[p++] = (byte) ((length >>> 16) & 0xff); 
-            b[p++] = (byte) ((length >>>  8) & 0xff); 
-            b[p++] = (byte) ((length       ) & 0xff);
-
-            currentBuffer.position   = p;
+			p = writeInt(length, b, p);
+            
+			currentBuffer.position   = p;
             currentBuffer.bytes_free -= total_log_record_length;
 		}
         else
         {
-            writeInt(length);
-            writeLong(instant);
-            write(data, data_offset, length - optional_data_length);
+			
+			/** Because current log record will never fit in a single buffer
+			 * a direct write to the log file is required instead of 
+			 * writing the log record through  the log bufffers. 
+			 */
+			directWrite = true;
+
+			byte[] b    = currentBuffer.buffer;
+            int    p    = currentBuffer.position;
+
+            // writeInt(length)
+			p = writeInt(length , b, p);
+            
+            // writeLong(instant)
+			p = writeLong(instant, b, p);
+
+			currentBuffer.position   = p;
+			currentBuffer.bytes_free -= LOG_RECORD_HEADER_SIZE;
+
+			/** using a seperate small buffer to write the traling length
+			 * instead of the log buffer because data portion will be 
+			 * written directly to log file after the log buffer is 
+			 * flushed and the trailing length should be written after that. 
+			 */
+
+			// writeInt(length)
+			writeInt(length , db, 0);
+
+			if(writeChecksum)
+			{
+				checksumLogOperation.reset();
+				checksumLogOperation.update(b, checksumLogRecordSize, p - checksumLogRecordSize);
+				checksumLogOperation.update(data, data_offset, length - optional_data_length);
+				if (optional_data_length != 0)
+				{
+					checksumLogOperation.update(optional_data, optional_data_offset, optional_data_length);	
+				}
+
+				// update the checksum to include the trailing length.
+				checksumLogOperation.update(db, 0, LOG_RECORD_TRAILER_SIZE);
+			
+				// write checksum log record to the log buffer 
+				writeChecksumLogRecord();
+			}
+			
+			
+			// now do the  writes directly to the log file. 
+
+			// flush all buffers before wrting directly to the log file. 
+			flushLogAccessFile();
+
+			// Note:No Special Synchronization required here , 
+			// There will be nothing to write by flushDirtyBuffers that can run
+			// in parallel to the threads that is executing this code. Above
+			// flush call should have written all the buffers and NO new log will 
+			// get added until the following direct log to file call finishes. 
+
+
+			// write the rest of the log directltly to the log file. 
+            writeToLog(data, data_offset, length - optional_data_length);
             if (optional_data_length != 0)
             {
-                write(
+                writeToLog(
                     optional_data, optional_data_offset, optional_data_length);
             }
-            writeInt(length);
+
+			// write the trailing length 
+			writeToLog(db,0, 4);
+			directWrite = false;
 		}
     }
 
 
-	public void writeInt(int i) throws IOException 
-    {
-		if (currentBuffer.bytes_free < 4)
-			switchLogBuffer();
-
-		byte[] b = currentBuffer.buffer;
-		int p = currentBuffer.position;
 
+	private final int writeInt(int i , byte b[], int p)
+	{
+	
         b[p++] = (byte) ((i >>> 24) & 0xff); 
         b[p++] = (byte) ((i >>> 16) & 0xff); 
         b[p++] = (byte) ((i >>> 8) & 0xff); 
-        b[p++] = (byte) (i & 0xff);
-
-		currentBuffer.position = p;
-		currentBuffer.bytes_free -= 4;
+        b[p++] = (byte) (i & 0xff);	
+		return p;
 	}
 
-	public void writeLong(long l) 
-        throws IOException 
-    {
-		if (currentBuffer.bytes_free < 8)
-			switchLogBuffer();
 
-		byte[] b = currentBuffer.buffer;
- 		int p = currentBuffer.position;
-        b[p++] = (byte) (((int)(l >>> 56)) & 0xff); 
+	private final int writeLong(long l , byte b[], int p)
+	{
+		b[p++] = (byte) (((int)(l >>> 56)) & 0xff); 
         b[p++] = (byte) (((int)(l >>> 48)) & 0xff); 
         b[p++] = (byte) (((int)(l >>> 40)) & 0xff); 
         b[p++] = (byte) (((int)(l >>> 32)) & 0xff); 
@@ -269,65 +382,60 @@
         b[p++] = (byte) (((int)(l >>> 16)) & 0xff); 
         b[p++] = (byte) (((int)(l >>> 8)) & 0xff); 
         b[p++] = (byte) (((int)l) & 0xff); 
-		currentBuffer.position = p;
+		return p;
+	}
+
+	public void writeInt(int i) 
+    {
+
+		if (SanityManager.DEBUG)
+		{
+			SanityManager.ASSERT(currentBuffer.bytes_free >= 4);
+		}
+		
+		currentBuffer.position = 
+			writeInt(i , currentBuffer.buffer, currentBuffer.position);
+		currentBuffer.bytes_free -= 4;
+	}
+
+	public void writeLong(long l) 
+    {
+		
+		if (SanityManager.DEBUG)
+		{
+			SanityManager.ASSERT(currentBuffer.bytes_free >= 8);
+		}
+		
+		currentBuffer.position = 
+			writeLong(l , currentBuffer.buffer, currentBuffer.position);
 		currentBuffer.bytes_free -= 8;
     }
 
 	public void write(int b) 
-        throws IOException 
     {
-
-		if (currentBuffer.bytes_free == 0)
-			switchLogBuffer();
-
+		if (SanityManager.DEBUG)
+		{
+			SanityManager.ASSERT(currentBuffer.bytes_free > 0);
+		}
+		
 		currentBuffer.buffer[currentBuffer.position++] = (byte) b;
 		currentBuffer.bytes_free--;
 	}
 
 
 	public void write(byte b[], int off, int len) 
-        throws IOException 
     {
-
-		if (len <= currentBuffer.bytes_free)  
-        {
-			// data fits in buffer
-			System.arraycopy(b, off, currentBuffer.buffer, currentBuffer.position, len);
-			currentBuffer.bytes_free -= len;
-			currentBuffer.position += len;
-			return;
+		if (SanityManager.DEBUG)
+		{
+			SanityManager.ASSERT(len <= currentBuffer.bytes_free);
 		}
-        else if (len <= currentBuffer.buffer.length) 
-        {
-            // some data will be cached
-            System.arraycopy(b, off, currentBuffer.buffer, currentBuffer.position, currentBuffer.bytes_free);
-            len -= currentBuffer.bytes_free;
-            off += currentBuffer.bytes_free;
-            currentBuffer.position += currentBuffer.bytes_free;
-            currentBuffer.bytes_free = 0;
-            switchLogBuffer();
-
-            System.arraycopy(b, off, currentBuffer.buffer, 0, len);
-            currentBuffer.position = len;
-            currentBuffer.bytes_free -= len;	
-        }
-        else
-        {
-			
-			//data will never fit in currentBuffer.buffer, write directly to log
-			//flush all buffers before wrting directly to the log file. 
-			flushLogAccessFile();
+		
+		System.arraycopy(b, off, currentBuffer.buffer, currentBuffer.position, len);
+		currentBuffer.bytes_free -= len;
+		currentBuffer.position += len;
+	}
 
-			//Note:No Special Synchronization required here , 
-			//There will be nothing to write by flushDirtyBuffers that can run
-			//in parallel to the threads that is executing this code. Above
-			//flush call should have written all the buffers and NO new log will 
-			//get added until the following direct log to file call finishes. 
 
-			writeToLog(b, off, len);
-			return;
-		}
-	}
     /**
      * Write data from all dirty buffers into the log file.
      * <p>
@@ -427,7 +535,7 @@
 
 
 	//flush all the the dirty buffers to disk
-	public void flushLogAccessFile() throws IOException 
+	public void flushLogAccessFile() throws IOException,  StandardException 
 	{
 		switchLogBuffer();
 		flushDirtyBuffers();
@@ -442,11 +550,24 @@
 	 * when  flushDirtyBuffers() is invoked by  a commit call 
 	 * or when no more free buffers are available. 
 	 */
-	public void switchLogBuffer() throws IOException  
+	public void switchLogBuffer() throws IOException, StandardException  
     {
 
 		synchronized(this)
 		{
+			// ignore empty buffer switch requests
+			if(currentBuffer.position == checksumLogRecordSize)
+				return;
+
+			// calculate the checksum for the current log buffer 
+			// and write the record to the space reserverd in 
+			// the beginning of the buffer. 
+			if(writeChecksum && !directWrite)
+			{
+				checksumLogOperation.reset();
+				checksumLogOperation.update(currentBuffer.buffer, checksumLogRecordSize, currentBuffer.position - checksumLogRecordSize);
+				writeChecksumLogRecord();
+			}
 
 			//add the current buffer to the flush buffer list
 			dirtyBuffers.addLast(currentBuffer);
@@ -467,13 +588,13 @@
 
 			//switch over to the next log buffer, let someone else write it.
 			currentBuffer = (LogAccessFileBuffer) freeBuffers.removeFirst();
-			currentBuffer.init();
-     
+			currentBuffer.init(checksumLogRecordSize);
+
 			if (SanityManager.DEBUG)
 			{
-				SanityManager.ASSERT(currentBuffer.position == 0);
+				SanityManager.ASSERT(currentBuffer.position == checksumLogRecordSize);
 				SanityManager.ASSERT(
-									 currentBuffer.bytes_free == currentBuffer.buffer.length);
+									 currentBuffer.bytes_free == currentBuffer.length);
                 SanityManager.ASSERT(currentBuffer.bytes_free > 0);
 			}
 		}
@@ -545,11 +666,11 @@
 		}
 	}
 
-	public void close() throws IOException 
+	public void close() throws IOException, StandardException
     {
 		if (SanityManager.DEBUG) 
         {
-			if (currentBuffer.position != 0)
+			if (currentBuffer.position !=  checksumLogRecordSize)
 				SanityManager.THROWASSERT(
 				"Log file being closed with data still buffered " + 
                 currentBuffer.position +  " " + currentBuffer.bytes_free);
@@ -572,6 +693,7 @@
 		{
             if (log != null)
             {
+
                 // Try to handle case where user application is throwing
                 // random interrupts at cloudscape threads, retry in the case
                 // of IO exceptions 5 times.  After that hope that it is 
@@ -602,6 +724,142 @@
 			mon_numBytesToLog += len;
 		}
 	}
+
+	/**
+	 * reserve the space for the checksum log record in the log file. 
+	 * @param  the length of the log record that is going to be written
+	 * @param  logFileNumber current log file number 
+	 * @param  currentPosition  current position in the log file. 
+	 * @return the space that is needed to write a checksum log record.
+	 */
+	protected long reserveSpaceForChecksum(int length, long logFileNumber, long currentPosition )
+		throws StandardException, IOException 
+	{
+		if(!writeChecksum)
+			return 0;
+
+		int total_log_record_length = length + LOG_RECORD_FIXED_OVERHEAD_SIZE;
+		boolean reserveChecksumSpace = false;
+		
+		/* checksum log record is calculated for a group of log 
+		 * records that can fit in to a single buffer or for 
+		 * a single record when it does not fit into 
+		 * a fit into a buffer at all. When a new buffer 
+		 * is required to write a log record, log space 
+		 * has to be reserved before writing the log record
+		 * becuase checksum is written in the before the 
+		 * log records that are being checksummed. 
+		 * What it also means is a real log instant has to be 
+		 * reserved for writing the checksum log record in addition 
+		 * to the log buffer space.
+		 */
+		
+
+		/* reserve checkum space for new log records if a log buffer switch had
+		 * happened before because of a explicit log flush requests(like commit)
+		 * or a long record write 
+		 */
+		if(currentBuffer.position == checksumLogRecordSize)
+		{
+			reserveChecksumSpace = true;
+		}
+		else{
+			if (total_log_record_length > currentBuffer.bytes_free)
+			{
+				// the log record that is going to be written is not 
+				// going to fit in the current buffer, switch the 
+				// log buffer to create buffer space for it. 
+				switchLogBuffer();
+				reserveChecksumSpace = true;
+			}
+		}
+		
+		if(reserveChecksumSpace)
+		{
+			if (SanityManager.DEBUG)
+			{
+				// Prevoiusly reserved real checksum instant should have been
+				// used, before an another one is generated. 
+				SanityManager.ASSERT(checksumInstant == -1,  "CHECKSUM INSTANT IS GETTING OVER WRITTEN");
+			}
+			
+			checksumInstant = LogCounter.makeLogInstantAsLong(logFileNumber, currentPosition);
+			return  checksumLogRecordSize;
+		}else
+		{
+			return 0 ;
+		}
+	}
+
+
+	/*
+	 * generate the checkum log record and write it into the log buffer.
+	 */
+	private void writeChecksumLogRecord() throws IOException, StandardException
+	{
+		
+		byte[] b    = currentBuffer.buffer;
+		int    p    = 0; //checksum is written in the beginning of the buffer
+
+		// writeInt(length)
+		p = writeInt(checksumLength, b , p);
+            
+		// writeLong(instant)
+		p = writeLong(checksumInstant, b , p);
+
+		//write the checksum log operation  
+		logOutputBuffer.setData(b);
+		logOutputBuffer.setPosition(p);
+		logicalOut.writeObject(checksumLogRecord);
+
+		if(databaseEncrypted)
+		{
+			//encrypt the checksum log operation part.
+			int len = 
+				logFactory.encrypt(b, LOG_RECORD_HEADER_SIZE, checksumLength, 
+								   b, LOG_RECORD_HEADER_SIZE);
+			
+		   
+			if (SanityManager.DEBUG)
+				SanityManager.ASSERT(len == checksumLength, 
+									 "encrypted log buffer length != log buffer len");
+		}
+
+		p = LOG_RECORD_HEADER_SIZE + checksumLength ;
+
+		// writeInt(length) trailing
+		p = writeInt(checksumLength, b, p );
+		
+		if (SanityManager.DEBUG)
+		{
+			SanityManager.ASSERT(p == checksumLogRecordSize, "position=" + p  + "ckrecordsize=" + checksumLogRecordSize);
+			if (SanityManager.DEBUG_ON(LogToFile.DBG_FLAG))
+			{
+				SanityManager.DEBUG(
+									LogToFile.DBG_FLAG, 
+									"Write log record: tranId=Null"  +
+									" instant: " + LogCounter.toDebugString(checksumInstant) + " length: " +
+									checksumLength + "\n" + checksumLogOperation + "\n");
+			}
+			checksumInstant = -1; 
+		}
+
+	}
+
+
+	protected void writeEndMarker(int marker) throws IOException, StandardException 
+	{
+		//flush all the buffers and then write the end marker.
+		flushLogAccessFile();
+		
+		byte[] b    = currentBuffer.buffer;
+		int    p    = 0; //end is written in the beginning of the buffer, no
+						 //need to checksum a int write.
+		p = writeInt(marker , b , p);
+		writeToLog(b, 0, p);
+	}
+
+	
 }
 
 

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFileBuffer.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFileBuffer.java?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFileBuffer.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogAccessFileBuffer.java Thu Mar 31 15:55:44 2005
@@ -38,6 +38,7 @@
     protected byte[]    buffer;
     protected int       bytes_free;
     protected int       position;
+	protected int       length;
 
     LogAccessFileBuffer next;
     LogAccessFileBuffer prev;
@@ -53,17 +54,18 @@
         prev        = null;
         next        = null;
 
-        init();
+        init(0);
     }
 
     /**************************************************************************
      * Private/Protected methods of This class:
      **************************************************************************
      */
-    public void init()
+    public void init(int reserve)
     {
-        bytes_free  = buffer.length;
-        position    = 0;
+		length =  buffer.length - reserve;
+        bytes_free  = length;
+        position    = reserve;
     }
 
     /**************************************************************************

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogRecord.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogRecord.java?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogRecord.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogRecord.java Thu Mar 31 15:55:44 2005
@@ -145,6 +145,21 @@
 		return tranId.getMaxStoredSize();
 	}
 
+	
+	public static int getStoredSize(int group, TransactionId xactId)
+	{
+		
+		if (SanityManager.DEBUG)
+		{
+			SanityManager.ASSERT(xactId == null, 
+								 "size calculation are based on xactId being  null"); 
+		}
+
+		return 	formatLength + 	CompressedNumber.sizeInt(group) + 
+			FormatIdUtil.getFormatIdByteLength(StoredFormatIds.NULL_FORMAT_ID);
+	}
+
+
 	public TransactionId getTransactionId() 
 		 throws IOException, ClassNotFoundException 
 	{
@@ -292,4 +307,7 @@
 	}
 
 
+	public boolean isChecksum()	{
+		return ((group & Loggable.CHECKSUM) != 0);
+	}
 }

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/LogToFile.java Thu Mar 31 15:55:44 2005
@@ -282,7 +282,7 @@
 
 	protected LogAccessFile logOut;		// an output stream to the log file
 								// (access of the variable should sync on this)
-
+	private   StorageRandomAccessFile firstLog = null;
 	protected long		     endPosition = -1; // end position of the current log file
 	long					 lastFlush = 0;	// the position in the current log
 											// file that has been flushed to disk
@@ -565,6 +565,12 @@
 
 		rawStoreFactory = rsf;
 		dataFactory     = df;
+		
+		// initialize the log writer only after the rawstorefactory is available, 
+		// log writer requires encryption block size info from rawstore factory 
+		// to encrypt checksum log records. 
+		if (firstLog != null) 
+			logOut = new LogAccessFile(this, firstLog, logBufferSize);
 
 		// we don't want to set ReadOnlyDB before recovery has a chance to look
 		// at the latest checkpoint and determine that the database is shutdown
@@ -1003,7 +1009,7 @@
 				}
 
 				if (theLog != null)
-					logOut = new LogAccessFile(theLog, logBufferSize);
+					logOut = new LogAccessFile(this, theLog, logBufferSize);
 				
 				if(logSwitchRequired)
 					switchLogFile();
@@ -1850,7 +1856,8 @@
 
 					// write out an extra 0 at the end to mark the end of the log
 					// file.
-					logOut.writeInt(0);
+					
+					logOut.writeEndMarker(0);
 
 					endPosition += 4;
 					//set that we are in log switch to prevent flusher 
@@ -1859,6 +1866,7 @@
 					// flush everything including the int we just wrote
 					flush(logFileNumber, endPosition);
 					
+					
 					// simulate out of log error after the switch over
 					if (SanityManager.DEBUG)
 					{
@@ -1883,7 +1891,7 @@
 						newLog.seek(endPosition);
 					}
 
-					logOut = new LogAccessFile(newLog, logBufferSize);
+					logOut = new LogAccessFile(this, newLog, logBufferSize);
 					newLog = null;
 
 
@@ -1972,7 +1980,7 @@
 		@exception IOException Failed to flush to the log
 	*/
 	private void flushBuffer(long fileNumber, long wherePosition)
-		throws IOException
+		throws IOException, StandardException
 	{
 		synchronized (this) {
 			if (fileNumber < logFileNumber)	// history
@@ -2913,29 +2921,28 @@
 					}
 
 					// don't need to try to delete it, we know it isn't there
-                    StorageRandomAccessFile theLog = privRandomAccessFile(logFile, "rw");
+                    firstLog = privRandomAccessFile(logFile, "rw");
 
-					if (!initLogFile(theLog, logFileNumber, LogCounter.INVALID_LOG_INSTANT))
+					if (!initLogFile(firstLog, logFileNumber, LogCounter.INVALID_LOG_INSTANT))
                     {
 						throw StandardException.newException(
                             SQLState.LOG_SEGMENT_NOT_EXIST, logFile.getPath());
                     }
 
-					endPosition = theLog.getFilePointer();
-					lastFlush = theLog.getFilePointer();
+					endPosition = firstLog.getFilePointer();
+					lastFlush = firstLog.getFilePointer();
 
                     //if write sync is true , prellocate the log file
                     //and reopen the file in rws mode.
                     if(isWriteSynced)
                     {
                         //extend the file by wring zeros to it
-                        preAllocateNewLogFile(theLog);
-                        theLog.close();
-                        theLog=  privRandomAccessFile(logFile, "rws");
+                        preAllocateNewLogFile(firstLog);
+                        firstLog.close();
+                        firstLog=  privRandomAccessFile(logFile, "rws");
                         //postion the log at the current log end postion
-                        theLog.seek(endPosition);
+                        firstLog.seek(endPosition);
                     }
-					logOut = new LogAccessFile(theLog, logBufferSize);
 
 					if (SanityManager.DEBUG)
 					{
@@ -2949,6 +2956,7 @@
 					// read only database
 					ReadOnlyDB = true;
 					logOut = null;
+					firstLog = null;
 				}
 
 				recoveryNeeded = false;
@@ -3012,8 +3020,9 @@
 				try {
 					logOut.flushLogAccessFile();
 					logOut.close();
-				} catch (IOException ioe) {
 				}
+				catch (IOException ioe) {}
+				catch(StandardException se){}
 				logOut = null;
 			}
 		}
@@ -3319,6 +3328,9 @@
                     }
 				}
 
+				//reserve the space for the checksum log record
+				endPosition += logOut.reserveSpaceForChecksum(length, logFileNumber,endPosition);
+
 				// don't call currentInstant since we are already in a
 				// synchronzied block 
 				instant = 
@@ -3793,6 +3805,22 @@
 		return rawStoreFactory.getEncryptionBlockSize();
 	}
 
+	/**
+	   returns the length that will make the data to be multiple of encryption
+	   block size based on the given length. Block cipher algorithms like DES 
+	   and Blowfish ..etc  require their input to be an exact multiple of the block size.
+	*/
+	public int getEncryptedDataLength(int length)
+	{
+		if ((length % getEncryptionBlockSize()) != 0)
+		{
+			return length + getEncryptionBlockSize() - (length % getEncryptionBlockSize());
+		}
+
+		return length;
+	}
+
+
 
 	/**
 	  Get the instant of the first record which was not
@@ -3939,16 +3967,23 @@
 			Monitor.logMessage("TEST_LOG_INCOMPLETE_LOG_WRITE: writing " + bytesToWrite + 
 				   " bytes out of " + length + " + " + LOG_RECORD_OVERHEAD + " log record");
 
-
-
-
-			long instant = currentInstant();
+			long instant;
 			try
 			{
+								
 				synchronized (this)
 				{
-						//check if the length of the records to be written is 
-						//actually smaller than the number of bytesToWrite 
+					// reserve the space for the checksum log record
+					// NOTE:  bytesToWrite include the log record overhead.
+					endPosition += 
+						logOut.reserveSpaceForChecksum(((length + LOG_RECORD_OVERHEAD) 
+														< bytesToWrite ? length :
+														(bytesToWrite - LOG_RECORD_OVERHEAD)),
+													   logFileNumber,endPosition);
+					instant = currentInstant();
+
+					//check if the length of the records to be written is 
+					//actually smaller than the number of bytesToWrite 
 					if(length + LOG_RECORD_OVERHEAD < bytesToWrite)
 						endPosition += (length + LOG_RECORD_OVERHEAD);
 					else
@@ -4073,6 +4108,25 @@
 		}	
 	}
 
+	/**
+	 * Get the log file to Simulate a log corruption 
+	 * FOR UNIT TESTING USAGE ONLY 
+	*/
+	public StorageRandomAccessFile getLogFileToSimulateCorruption(long filenum) throws IOException, StandardException
+	{
+		if (SanityManager.DEBUG)
+		{
+			//long filenum = LogCounter.getLogFileNumber(logInstant);
+			//			long filepos = LogCounter.getLogFilePosition(logInstant);
+			StorageFile fileName = getLogFileName(filenum);
+			StorageRandomAccessFile log = null;
+			return privRandomAccessFile(fileName, "rw");
+		}
+		
+		return null;
+
+	}
+	
 
 	/*********************************************************************
 	 * Log Testing
@@ -4120,9 +4174,6 @@
 	  simulated to be full.
 	*/
 	public static final String TEST_RECORD_TO_FILL_LOG = SanityManager.DEBUG ? "db2j.unittest.recordToFillLog" : null;
-
-
-
 
 
 	//enable the log archive mode

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/Scan.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/Scan.java?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/Scan.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/log/Scan.java Thu Mar 31 15:55:44 2005
@@ -39,6 +39,7 @@
 import org.apache.derby.io.StorageRandomAccessFile;
 
 import java.io.IOException;
+import org.apache.derby.iapi.store.raw.Loggable;
 
 /**
 
@@ -460,9 +461,20 @@
 			}
 
 			lr = (LogRecord) input.readObject();
-			if (groupmask != 0 || tranId != null)
+
+			// skip the checksum log records, there is no need to look at them 
+			// during backward scans. They are used only in forwardscan during recovery. 
+			if(lr.isChecksum())
 			{
-				if (groupmask != 0 && (groupmask & lr.group()) == 0)
+				candidate = false; 
+			}else if (groupmask != 0 || tranId != null)
+			{
+
+				// skip the checksum log records  
+				if(lr.isChecksum())
+					candidate = false; 
+
+				if (candidate && groupmask != 0 && (groupmask & lr.group()) == 0)
 					candidate = false; // no match, throw this log record out 
 
 				if (candidate && tranId != null)
@@ -950,6 +962,93 @@
 
 			// the scan is now positioned just past this log record and right
 			// at the beginning of the next log record
+
+
+			/** if the current log record is a checksum log record then
+			 * using the information available in this record validate
+			 * that data in the log file by matching the checksum in 
+			 * checksum log record and by recalculating the checksum for the 
+			 * specified length of the data in the log file. cheksum values
+			 * should match unless the right was incomplete before the crash.
+			 */
+			if(lr.isChecksum())
+			{
+				// checksum log record should not be returned to the logger recovery redo
+				// routines, it is just used to identify the incomplete log writes.
+
+				candidate = false;
+				Loggable op = lr.getLoggable(); 
+				if (SanityManager.DEBUG)
+                {
+                    if (SanityManager.DEBUG_ON(LogToFile.DUMP_LOG_ONLY) ||
+                        SanityManager.DEBUG_ON(LogToFile.DBG_FLAG))
+
+						SanityManager.DEBUG(LogToFile.DBG_FLAG, 
+											"scanned " + "Null" + " : " + op + 
+											" instant = " + 
+											LogCounter.toDebugString(currentInstant) + 
+											" logEnd = " +  LogCounter.toDebugString(knownGoodLogEnd));
+				}
+
+				ChecksumOperation clop = (ChecksumOperation) op;
+				int ckDataLength =  clop.getDataLength(); 
+				// resize the buffer to be size of checksum data length if required.
+				if (data.length < ckDataLength)
+				{
+					// make a new array of sufficient size and reset the arrary
+					// in the input stream
+					data = new byte[ckDataLength];
+					input.setData(data);
+					input.setLimit(0, ckDataLength);
+				}
+				
+				boolean validChecksum = false;
+				// check if the expected number of bytes by the checksum log
+				// record actually exist in the file and then verify if checksum
+				// is valid to identify any incomplete out of order writes.
+				if((recordStartPosition + ckDataLength) <= currentLogFileLength)
+				{
+					// read the data into the buffer
+					scan.readFully(data, 0, ckDataLength);
+					// verify the checksum 
+					if(clop.isChecksumValid(data, 0 , ckDataLength))
+						validChecksum = true;
+				}
+
+
+				if(!validChecksum)
+				{
+					// declare that the end of the transaction log is fuzzy, checksum is invalid
+					// only when the writes are incomplete; this can happen
+					// only when writes at the end of the log were partially
+					// written before the crash. 
+
+					if (SanityManager.DEBUG)
+                    {
+                        if (SanityManager.DEBUG_ON(LogToFile.DBG_FLAG))
+                        {
+                            SanityManager.DEBUG(
+                                LogToFile.DBG_FLAG, 
+                                "detected fuzzy log end on log file while doing checksum checks " + 
+								currentLogFileNumber + 
+                                " checksum record start position " + recordStartPosition + 
+                                " file length " + currentLogFileLength + 
+								" checksumDataLength=" + ckDataLength);
+                        }
+						
+					}
+					
+					fuzzyLogEnd = true;
+					scan.close();
+					scan = null;
+					return null;
+				}
+
+				// reset the scan to the start of the next log record
+				scan.seek(recordStartPosition);
+			}
+
+
 		} while (candidate == false) ;
 
 		return lr;

Modified: incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storeunit.runall
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storeunit.runall?view=diff&r1=159650&r2=159651
==============================================================================
--- incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storeunit.runall (original)
+++ incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storeunit.runall Thu Mar 31 15:55:44 2005
@@ -1,26 +1,34 @@
-unit/T_RawStoreFactory.unit
-unit/T_FileSystemData.unit
-unit/T_StreamFile.unit
-unit/T_AccessFactory.unit
-unit/T_Heap.unit
-unit/T_b2i.unit
-unit/T_SortController.unit
-unit/TLockFactory.unit
-unit/TUUIDFactory.unit
-unit/recoverySetup.unit
-unit/recoveryTest.unit
-unit/recoverBadLogSetup.unit
-unit/recoverBadLog1.unit
-unit/recoverBadLog2.unit
-unit/recoverBadLog3.unit
-unit/recoverBadLog4.unit
-unit/recoverBadLog5.unit
-unit/recoverBadLog6.unit
-unit/recoverBadLog7.unit
-unit/recoverBadLog1.unit
-unit/recoverBadLog2.unit
-unit/fillLog.unit
-unit/logSwitchFail.unit
-unit/fullRecoveryFail.unit
-unit/fullRecoveryFail.unit
-unit/fullRecovery.unit
+unit/T_RawStoreFactory.unit
+unit/T_FileSystemData.unit
+unit/T_StreamFile.unit
+unit/T_AccessFactory.unit
+unit/T_Heap.unit
+unit/T_b2i.unit
+unit/T_SortController.unit
+unit/TLockFactory.unit
+unit/TUUIDFactory.unit
+unit/recoverySetup.unit
+unit/recoveryTest.unit
+unit/recoverBadLogSetup.unit
+unit/recoverBadLog1.unit
+unit/recoverBadLog2.unit
+unit/recoverBadLog3.unit
+unit/recoverBadLog4.unit
+unit/recoverBadLog5.unit
+unit/recoverBadLog6.unit
+unit/recoverBadLog7.unit
+unit/recoverBadLog1.unit
+unit/recoverBadLog2.unit
+unit/fillLog.unit
+unit/logSwitchFail.unit
+unit/fullRecoveryFail.unit
+unit/fullRecoveryFail.unit
+unit/fullRecovery.unit
+unit/recoverBadChecksumLogSetup.unit
+unit/recoverBadChecksumLog1.unit
+unit/recoverBadChecksumLog2.unit
+unit/recoverBadChecksumLog3.unit
+unit/recoverBadChecksumLog4.unit
+unit/recoverBadChecksumLog5.unit
+unit/recoverBadChecksumLog6.unit
+unit/recoverBadChecksumLog7.unit