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/05/11 02:13:09 UTC

svn commit: r169538 - in /incubator/derby/code/trunk/java: engine/org/apache/derby/iapi/reference/ engine/org/apache/derby/impl/store/raw/data/ engine/org/apache/derby/impl/store/raw/log/ engine/org/apache/derby/loc/ testing/org/apache/derbyTesting/functionTests/master/ testing/org/apache/derbyTesting/functionTests/suites/ testing/org/apache/derbyTesting/functionTests/tests/store/

Author: mikem
Date: Tue May 10 17:13:08 2005
New Revision: 169538

URL: http://svn.apache.org/viewcvs?rev=169538&view=rev
Log:
committing patch for DERBY-218, relaxed durability option - 
for ksunithaghm@gmail.com.

A little background: Sometime earlier on the list, Dan posted a fix to make derby go faster with relaxed durability with some flags.  The post is at http://article.gmane.org/gmane.comp.apache.db.derby.user/681/match=relaxed+durability
This mode is very useful for unit testing or at development time when recoverability is not required.
Basically in this mode, syncs are disabled for logs, data writes at checkpoint, page allocation when file is grown; - what this means is that data is not flushed all the way to the disk and in most cases i/o cost is not involved. Note,  code already exists in Derby to do this.
So for Derby 218, This  patch addresses the following requirements:  1) Have a single property to enable this relaxed durability mode, so it is easy for users  to enable it.
2) Print message to derby.log that this mode is enabled
3) A way to report boot errors that may be because of this option being enabled. What this maps to is - have a marker to recognize that database was booted in this mode, and then on subsequent boot; if errors happen during recovery - report a message that it could have happened because of this mode being set. 

Thanks Suresh for your comments.  I have attached a new patch that takes care of them and as well as the property name change as discussed on the list.

So now,
1)To enable this no syncs mode, the property setting  is derby.system.durability=test  If it is set to any other value other than test, then this property setting is ignored and the system will default to the usual mode where all syncs happen correctly.
2) If database is booted in this mode then the  flag  to indicate this mode is being used is set in log.ctrl file and it will not change after that.Thus there wont be any switching between this mode ( no syncs) to default ( all syncs) mode because we cannot guarantee that the database will be in a consistent state once its booted in this no sync mode..
3) This  flag is now written out as part of the log control flags in log.ctrl file instead of using a spare byte. 


Added:
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/TestDurabilityProperty.out
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/TestDurabilityProperty.java
Modified:
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/MessageId.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/Property.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
    incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.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/loc/messages_en.properties
    incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemats.runall

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/MessageId.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/MessageId.java?rev=169538&r1=169537&r2=169538&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/MessageId.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/MessageId.java Tue May 10 17:13:08 2005
@@ -58,7 +58,10 @@
     String LOG_CANNOT_CREATE_NEW_DUETO      = "L017";
     String LOG_MAYBE_INCONSISTENT           = "L018";
     String LOG_TRANS_LOG_SCAN               = "L019";
-
+    String LOG_WAS_IN_DURABILITY_TESTMODE_NO_SYNC = "L020"; // database was running in
+                                                            // derby.system.durability set to test 
+    String LOG_DURABILITY_TESTMODE_NO_SYNC_ERR = "L021"; // hint that error could be because 
+                                                         // derby.system.durability was set to test
 
     /*
      * Raw Store data
@@ -75,6 +78,8 @@
     String STORE_BACKUP_ABORTED             = "D010";
     String STORE_REMOVED_BACKUP             = "D011";
     String STORE_BACKUP_COMPLETED           = "D012";
+    String STORE_DURABILITY_TESTMODE_NO_SYNC = "D013"; // for derby.system.durability is 
+                                                       // set to test
 
 
 	/*

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/Property.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/Property.java?rev=169538&r1=169537&r2=169538&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/Property.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/iapi/reference/Property.java Tue May 10 17:13:08 2005
@@ -368,55 +368,62 @@
 		<P>
 	*/
 	String STORAGE_TEMP_DIRECTORY = "derby.storage.tempDirectory";
-
+	
     /**
-     * derby.storage.dataNotSyncedAtCheckPoint
+     * derby.system.durability
      * <p>
-     * When set, the store system will not force a sync() call on the
-     * containers during a checkpoint.
-     * <p>
-     * An internal debug system only flag.  The recovery system will not
-     * work properly if this flag is enabled, it is provided to do performance
-     * debugging to see whether the system is I/O bound based on checkpoint
-     * synchronous I/O.
-     * <p>
-     *
-     **/
-	public static final String STORAGE_DATA_NOT_SYNCED_AT_CHECKPOINT = 
-        "derby.storage.dataNotSyncedAtCheckPoint";
-
+     * Currently the only valid supported case insensitive value is 'test' 
+     * Note, if this property is set to any other value other than 'test', this 
+     * property setting is ignored
+     * 
+     * In the future, this property can be used to set different modes - for 
+     * example a form of relaxed durability where database can recover to a 
+     * consistent state, or to enable some kind of in-memory mode.
+     * <BR>
+     * When set to 'test', the store system will not force sync calls in the 
+     * following cases  
+     * - for the log file at each commit
+     * - for the log file before data page is forced to disk
+     * - for page allocation when file is grown
+     * - for data writes during checkpoint
+     * 
+     * That means
+     * - a commit no longer guarantees that the transaction's modification
+     *   will survive a system crash or JVM termination
+     * - the database may not recover successfully upon restart
+     * - a near full disk at runtime may cause unexpected errors
+     * - database can be in an inconsistent state
+     * <p>
+     * This setting is provided for performance reasons and should ideally
+     * only be used when the system can withstand the above consequences.
+     * <BR> 
+     * One sample use would be to use this mode (derby.system.durability=test)
+     * when using Derby as a test database, where high performance is required
+     * and the data is not very important
+     * <BR>
+     * Valid supported values are test
+     * <BR>
+     * Example
+     * derby.system.durability=test
+     * One can set this as a command line option to the JVM when starting the
+     * application or in the derby.properties file. It is a system level 
+     * property.
+     * <BR>
+     * This property is static; if you change it while Derby is running, 
+     * the change does not take effect until you reboot.  
+     */
+	public static final String DURABILITY_PROPERTY = 
+        "derby.system.durability";
+ 	
     /**
-     * derby.storage.dataNotSyncedAtAllocation
-     * <p>
-     * When set, the store system will not force a sync() call on the
-     * containers when pages are allocated.
-     * <p>
-     * An internal debug system only flag.  The recovery system will not
-     * work properly if this flag is enabled, it is provided to do performance
-     * debugging to see whether the system is I/O bound based on page allocation
-     * synchronous I/O.
-     * <p>
-     *
-     **/
-	public static final String STORAGE_DATA_NOT_SYNCED_AT_ALLOCATION = 
-        "derby.storage.dataNotSyncedAtAllocation";
-
-    /**
-     * derby.storage.logNotSynced
-     * <p>
-     * When set, the store system will not force a sync() call on the log at 
-     * commit.
-     * <p>
-     * An internal debug system only flag.  The recovery system will not
-     * work properly if this flag is enabled, it is provided to do performance
-     * debugging to see whether the system is I/O bound based on log file
-     * synchronous I/O.
-     * <p>
-     *
-     **/
-	public static final String STORAGE_LOG_NOT_SYNCED = 
-        "derby.storage.logNotSynced";
-
+     * This is a value supported for derby.system.durability
+     * When derby.system.durability=test, the storage system does not
+     * force syncs and the system may not recover. It is also possible that
+     * the database might be in an inconsistent state
+     * @see #DURABILITY_PROPERTY
+     */
+    public static final String DURABILITY_TESTMODE_NO_SYNC = "test";
+    
 	/**
      * derby.storage.fileSyncTransactionLog
      * <p>

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java?rev=169538&r1=169537&r2=169538&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java Tue May 10 17:13:08 2005
@@ -177,13 +177,13 @@
 	private StorageFile exFileLock; //file handle to get exclusive lock
 	private HeaderPrintWriter istream;
 	private static final String LINE = 
-	"----------------------------------------------------------------";
+        "----------------------------------------------------------------";
 
-    // debug only flag - disable syncing of data during page allocation.
-    boolean dataNotSyncedAtAllocation;
+    // disable syncing of data during page allocation.
+    boolean dataNotSyncedAtAllocation = false;
 
-    // debug only flag - disable syncing of data during checkpoint.
-    boolean dataNotSyncedAtCheckpoint;
+    // disable syncing of data during checkpoint.
+    boolean dataNotSyncedAtCheckpoint = false;
 
 	// these fields can be accessed directly by subclasses if it needs a
 	// different set of actions
@@ -375,29 +375,25 @@
 
 		droppedTableStubInfo = new Hashtable();
 
-        if (Performance.MEASURE)
+        // If derby.system.durability=test then set flags to disable sync of
+        // data pages at allocation when file is grown, disable sync of data
+        // writes during checkpoint
+        if (Property.DURABILITY_TESTMODE_NO_SYNC.equalsIgnoreCase(
+            PropertyUtil.getSystemProperty(Property.DURABILITY_PROPERTY)))
         {
-            // debug only flag - disable syncing of data during checkpoint.
-            dataNotSyncedAtCheckpoint = 
-                PropertyUtil.getSystemBoolean(
-                    Property.STORAGE_DATA_NOT_SYNCED_AT_CHECKPOINT);
-
-            // debug only flag - disable syncing of data during page allocation.
-            dataNotSyncedAtAllocation = 
-                PropertyUtil.getSystemBoolean(
-                    Property.STORAGE_DATA_NOT_SYNCED_AT_ALLOCATION);
-
-            if (dataNotSyncedAtCheckpoint)
-                Monitor.logMessage(
-                    "Warning: " + 
-                    Property.STORAGE_DATA_NOT_SYNCED_AT_CHECKPOINT +
-                    "set to true.");
-
-            if (dataNotSyncedAtAllocation)
-                Monitor.logMessage(
-                    "Warning: " + 
-                    Property.STORAGE_DATA_NOT_SYNCED_AT_ALLOCATION +
-                    "set to true.");
+            // - disable syncing of data during checkpoint.
+            dataNotSyncedAtCheckpoint = true;
+
+            // - disable syncing of data during page allocation.
+            dataNotSyncedAtAllocation = true;
+
+            // log message stating that derby.system.durability
+            // is set to a mode, where syncs wont be forced and the
+            // possible consequences of setting this mode
+            Monitor.logMessage(MessageService.getTextMessage(
+            	MessageId.STORE_DURABILITY_TESTMODE_NO_SYNC,
+            	Property.DURABILITY_PROPERTY,
+                Property.DURABILITY_TESTMODE_NO_SYNC));
 		}
 
         fileHandler = new RFResource( this);

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java?rev=169538&r1=169537&r2=169538&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java Tue May 10 17:13:08 2005
@@ -325,15 +325,8 @@
 				dataFactory.writeInProgress();
 				try
 				{
-                    if (Performance.MEASURE)
-                    {
-                        if (!dataFactory.dataNotSyncedAtAllocation)
-                            fileData.sync( false);
-                    }
-                    else
-                    {
+                    if (!dataFactory.dataNotSyncedAtAllocation)
                         fileData.sync( false);
-                    }
 				}
 				finally
 				{
@@ -504,16 +497,9 @@
 					dataFactory.writeInProgress();
 					inwrite = true;
 
-                    if (Performance.MEASURE)
-                    {
-                        if (!dataFactory.dataNotSyncedAtAllocation)
-                            fileData.sync( false);
-                    }
-                    else
-                    {
-                        fileData.sync( false);
-                    }
-				}
+                    if (!dataFactory.dataNotSyncedAtAllocation)
+                        fileData.sync(false);
+  				}
 				catch (IOException ioe)
 				{
 					// The disk may have run out of space. 
@@ -628,15 +614,9 @@
 			dataFactory.writeInProgress();
 			try
 			{
-                if (Performance.MEASURE)
-                {
-                    if (!dataFactory.dataNotSyncedAtCheckpoint)
-                        file.sync( false);
-                }
-                else
-                {
-                    file.sync( false);
-                }
+                if (!dataFactory.dataNotSyncedAtCheckpoint)
+                   file.sync(false);
+
 			}
 			finally
 			{

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?rev=169538&r1=169537&r2=169538&view=diff
==============================================================================
--- 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 Tue May 10 17:13:08 2005
@@ -265,7 +265,32 @@
 
 	/* Log Control file flags. */
 	private static final byte IS_BETA_FLAG = 0x1;
-
+	
+    /**
+     * When the derby.system.durability property is set to 'test', the store 
+     * system will not force sync calls in the following cases
+     * - for the log file at each commit
+     * - for the log file before data page is forced to disk
+     * - for page allocation when file is grown
+     * - for data writes during checkpoint
+     * This means it is possible that the recovery system may not work properly,
+     * committed transactions may be lost, and/or database may not
+     * be in a consistent state.
+     * In order that we recognize this case that the database was previously
+     * at any time booted in this mode, this value is written out
+     * into the log control file. This will help prevent us from 
+     * wasting time to resolve issues in such cases. 
+     * @see org.apache.derby.iapi.reference.Property#DURABILITY_PROPERTY
+     * This value is written as part of the log control file flags byte.
+     */
+    private static final byte IS_DURABILITY_TESTMODE_NO_SYNC_FLAG = 0x2;
+	
+    /**
+     * keeps track of if the database was booted previously at any time with 
+     * derby.system.durability=test
+     */
+    private static boolean wasDBInDurabilityTestModeNoSync = false;
+    
 	/* to err on the conservative side, unless otherwise set, assume log
 	 *	archive is ON 
 	 */
@@ -362,7 +387,7 @@
 	// log goes to the log subdirectory underneath the data directory
 	String logDevice;
 
-    // debug only flag - disable syncing of log file for debugging performance.
+    // disable syncing of log file when running in derby.system.durability=test
     private boolean logNotSynced = false;
 
 	private boolean logArchived = false;
@@ -403,8 +428,7 @@
 	
 	private CRC32 checksum = new CRC32(); // holder for the checksum
 
-
-	
+ 	
 	/**
 	 * Note: Why logging system support file sync and write sync ?
 	 * Note : The reason to support file and write sync of logs is 
@@ -1194,8 +1218,8 @@
 			tf.resetTranId();
 		}
 
-		// done with recovery
-
+        // done with recovery        
+        
 		/////////////////////////////////////////////////////////////
 		// setup checktpoint daemon
 		/////////////////////////////////////////////////////////////
@@ -1207,6 +1231,7 @@
         }
 	}
 
+ 
 	/**
 		Checkpoint the rawStore.
 
@@ -2136,16 +2161,31 @@
 		// For 2.0 beta we added the build number and the isBeta indication.
 		// (5 bytes from our first spare long)
 		daos.writeInt(jbmsVersion.getBuildNumberAsInt());
+
 		byte flags = 0;
-		if (onDiskBeta) flags |= IS_BETA_FLAG;
-		daos.writeByte(flags);
+		if (onDiskBeta) 
+            flags |= IS_BETA_FLAG;
+        
+        // When database is booted with derby.system.durability=test,
+        // this mode does not guarantee that 
+        // - database will recover 
+        // - committed transactions will not be lost
+        // - database will be in a consistent state
+        // Hence necessary to keep track of this state so we don't 
+        // waste time resolving issues in such cases.
+        // wasDBInDurabilityTestModeNoSync has information if database was
+        // previously booted at any time in this mode
+        if (logNotSynced || wasDBInDurabilityTestModeNoSync)
+            flags |= IS_DURABILITY_TESTMODE_NO_SYNC_FLAG;
+ 		daos.writeByte(flags);
 
 		//
 		// write some spare bytes after 2.0 we have 3 + 2(8) spare bytes.
-		long spare = 0;
-		daos.writeByte(0);
+ 		long spare = 0;
+       
 		daos.writeByte(0);
 		daos.writeByte(0);
+        daos.writeByte(0);
 		daos.writeLong(spare);
 		daos.flush();
 		// write the checksum for the control data written
@@ -2281,9 +2321,24 @@
 				onDiskMinorVersion = dais.readInt();
 				int dbBuildNumber = dais.readInt();
 				int flags = dais.readByte();
+				
+				// check if the database was booted previously at any time with
+                // derby.system.durability=test mode
+                // If yes, then on a boot error we report that this setting is
+                // probably the cause for the error and also log a warning
+                // in the derby.log that this mode was set previously
+                wasDBInDurabilityTestModeNoSync = 
+                    (flags & IS_DURABILITY_TESTMODE_NO_SYNC_FLAG) != 0;
 
+                if (SanityManager.DEBUG) {
+                    if (SanityManager.DEBUG_ON(LogToFile.DBG_FLAG))
+                        SanityManager.DEBUG(LogToFile.DBG_FLAG,
+                        "log control file, was derby.system.durability set to test = " +
+                        wasDBInDurabilityTestModeNoSync);
+                }
+                    
+				
 				onDiskBeta = (flags & IS_BETA_FLAG) != 0;
-
 				if (onDiskBeta)
 				{
 					// if is beta, can only be booted by exactly the same
@@ -2794,19 +2849,17 @@
 			isWriteSynced = false;
 		}
 
-		if (Performance.MEASURE)
-		{
-			// debug only flag - disable syncing of log.
-			logNotSynced = 
-				PropertyUtil.getSystemBoolean(Property.STORAGE_LOG_NOT_SYNCED);
-
-			if (logNotSynced)
-			{
-				Monitor.logMessage("logNotSynced = true");
-				//if log is Not being synced;files should not be open in write sync mode
-				isWriteSynced = false;
-			}
-			
+
+        // If derby.system.durability=test is set,then set flag to 
+        // disable sync of log records at commit and log file before 
+        // data page makes it to disk
+        if (Property.DURABILITY_TESTMODE_NO_SYNC.equalsIgnoreCase(
+               PropertyUtil.getSystemProperty(Property.DURABILITY_PROPERTY)))
+        {
+		    // disable syncing of log.
+		    logNotSynced = true;
+  		    //if log not being synced;files shouldn't be open in write sync mode
+		    isWriteSynced = false;	
 		}
 
 		// try to access the log
@@ -2829,11 +2882,30 @@
 			{
                 if (privExists(logControlFileName))
 				{
-					checkpointInstant = readControlFile(logControlFileName, startParams);
+					checkpointInstant = 
+                        readControlFile(logControlFileName, startParams);
+
+					// in case system was running previously with 
+                    // derby.system.durability=test then print a message 
+                    // to the derby log
+                    if (wasDBInDurabilityTestModeNoSync)
+                    {
+                        // print message stating that the database was
+                        // previously atleast at one time running with
+                        // derby.system.durability=test mode
+                        Monitor.logMessage(MessageService.getTextMessage(
+			           		MessageId.LOG_WAS_IN_DURABILITY_TESTMODE_NO_SYNC,
+			           		Property.DURABILITY_PROPERTY,
+                            Property.DURABILITY_TESTMODE_NO_SYNC));
+                    }
+						
 					if (checkpointInstant == LogCounter.INVALID_LOG_INSTANT &&
 										getMirrorControlFileName().exists())
+                    {
 						checkpointInstant =
-									readControlFile(getMirrorControlFileName(), startParams);
+                            readControlFile(
+                                getMirrorControlFileName(), startParams);
+                    }
 
 				}
 				else if (logDevice != null)
@@ -2879,13 +2951,14 @@
 					// blow away the log file if possible
                     if (!privDelete(logFile) && logFileNumber == 1)
                     {
+                        logErrMsgForDurabilityTestModeNoSync();
 						throw StandardException.newException(
                             SQLState.LOG_INCOMPATIBLE_FORMAT, dataDirectory);
                     }
 
-					// If logFileNumber > 1, we are not going to write that file just
-					// yet.  Just leave it be and carry on.  Maybe when we get there it
-					// can be deleted.
+					// If logFileNumber > 1, we are not going to write that 
+                    // file just yet.  Just leave it be and carry on.  Maybe 
+                    // when we get there it can be deleted.
 
 					createNewLog = true;
 				}
@@ -2913,6 +2986,7 @@
 
                         if (!privDelete(logFile))
                         {
+                            logErrMsgForDurabilityTestModeNoSync();
 							throw StandardException.newException(
                                     SQLState.LOG_INCOMPATIBLE_FORMAT,
                                     dataDirectory);
@@ -3576,7 +3650,7 @@
 			if (Performance.MEASURE)
 				mon_syncCalls++;
 
-			if(isWriteSynced)
+			if (isWriteSynced)
 			{
 				//LogAccessFile.flushDirtyBuffers() will allow only one write
 				//sync at a time, flush requests will get queued 
@@ -3584,15 +3658,8 @@
 			}
 			else
 			{
-				if (Performance.MEASURE)
-				{
-					if (!logNotSynced)
-						logOut.syncLogAccessFile();
-				}
-				else
-				{
-					logOut.syncLogAccessFile();
-				}
+				if (!logNotSynced)
+				    logOut.syncLogAccessFile();
 			}
 
 			syncSuceed = true;
@@ -3925,6 +3992,7 @@
 	*/
 	protected void logErrMsg(String msg)
 	{
+       	logErrMsgForDurabilityTestModeNoSync();
 		Monitor.logTextMessage(MessageId.LOG_BEGIN_ERROR);
 		Monitor.logMessage(msg);
 		Monitor.logTextMessage(MessageId.LOG_END_ERROR);
@@ -3936,6 +4004,7 @@
 	*/
 	protected void logErrMsg(Throwable t)
 	{
+		logErrMsgForDurabilityTestModeNoSync();
 		if (corrupt != null)
 		{
 			Monitor.logTextMessage(MessageId.LOG_BEGIN_CORRUPT_STACK);
@@ -3950,6 +4019,28 @@
 			Monitor.logTextMessage(MessageId.LOG_END_ERROR_STACK);
 		}
 	}
+
+
+    /**
+     * In case of boot errors, and if database is either booted
+     * with derby.system.durability=test or was previously at any time booted in
+     * this mode, mention in the error message that the error is probably 
+     * because the derby.system.durability was set. 
+     * Dont want to waste time to resolve issues in such
+     * cases
+     * <p>
+     * MT - not needed, informational only
+     */
+    private void logErrMsgForDurabilityTestModeNoSync()
+    {
+        if (logNotSynced || wasDBInDurabilityTestModeNoSync)
+        {
+            Monitor.logTextMessage(
+                MessageId.LOG_DURABILITY_TESTMODE_NO_SYNC_ERR,
+                Property.DURABILITY_PROPERTY,
+                Property.DURABILITY_TESTMODE_NO_SYNC);
+        }
+    }
 
     /**
      * print stack trace from the Throwable including

Modified: incubator/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties?rev=169538&r1=169537&r2=169538&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties (original)
+++ incubator/derby/code/trunk/java/engine/org/apache/derby/loc/messages_en.properties Tue May 10 17:13:08 2005
@@ -1150,6 +1150,9 @@
 L016=cannot create a new log file {0}
 L017=cannot create a new log file {0} due to {1}
 L018=System may be in a inconsistent state, missing file {0}
+L020=WARNING: The database was at least at one time booted with {0}={1}. As a result of this mode being used, it is possible that the database may not be able to recover, committed transactions may be lost, and the database may be in an inconsistent state. This may cause unexpected errors.\n
+L021=WARNING: The database was either previously booted or is currently booted with {0} set to {1}. Note that this mode {0}={1} does not guarantee recoverability, and is probably what might have caused this boot error.\n
+
 
 # Class Manager
 C000=Wrote class {0} to file {1}. Please provide support with the file and the following exception information: {2}
@@ -1174,6 +1177,7 @@
 D010=Backup aborted due to error
 D011=removed old backup copy at {0}
 D012=Backup completed, log instant at {0} \n
+D013=WARNING: The database is booted with {0}={1}. In this mode, it is possible that database may not be able to recover, committed transactions may be lost, database may be in an inconsistent state. Please use this mode only when these consequences are acceptable  \n
 
 # Connectivity 
 J004=database identity

Added: incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/TestDurabilityProperty.out
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/TestDurabilityProperty.out?rev=169538&view=auto
==============================================================================
--- incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/TestDurabilityProperty.out (added)
+++ incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/master/TestDurabilityProperty.out Tue May 10 17:13:08 2005
@@ -0,0 +1,8 @@
+1. create database with derby.system.durability=test mode
+num successful inserts = 500
+Is warning message about derby.system.durability=test present in derby.log =true
+log.ctrl file has durability testMode no sync marker value = true
+2. shutdown database and reboot database without derby.system.durability=test and test for marker in log.ctrl file
+expected exception for shutdown.Derby system shutdown.
+log.ctrl file has durability testMode no sync marker value = true
+Is warning message about derby.system.durability=test present in derby.log =true

Modified: incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemats.runall
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemats.runall?rev=169538&r1=169537&r2=169538&view=diff
==============================================================================
--- incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemats.runall (original)
+++ incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/suites/storemats.runall Tue May 10 17:13:08 2005
@@ -4,3 +4,4 @@
 store/longColumn.sql
 store/madhare.sql
 store/streamingColumn.java
+store/TestDurabilityProperty.java

Added: incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/TestDurabilityProperty.java
URL: http://svn.apache.org/viewcvs/incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/TestDurabilityProperty.java?rev=169538&view=auto
==============================================================================
--- incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/TestDurabilityProperty.java (added)
+++ incubator/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/TestDurabilityProperty.java Tue May 10 17:13:08 2005
@@ -0,0 +1,238 @@
+/*
+ 
+ Derby - Class org.apache.derbyTesting.functionTests.tests.store.TestNoSyncs
+ 
+ Copyright 2002, 2005 The Apache Software Foundation or its licensors, as applicable.
+ 
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+ 
+ http://www.apache.org/licenses/LICENSE-2.0
+ 
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ 
+ */
+
+package org.apache.derbyTesting.functionTests.tests.store;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.sql.PreparedStatement;
+import java.sql.DriverManager;
+import java.io.*;
+
+/**
+ * This program tests the system when the derby.system.durability property is
+ * set to 'test'. 
+ * <BR>
+ * When the derby.system.durability is set to 'test', the system will not do 
+ * any sync to the
+ * disk and the recovery system will not work property. It is provided for
+ * performance reasons and should ideally only be used when there is no
+ * requirement for the database to recover.
+ * <p>
+ * When set, the system will not do any syncs, the cases namely - no sync of the
+ * log file at each commit - no sync of the log file before data page is forced
+ * to disk - no sync of page allocation when file is grown - no sync of data
+ * writes during checkpoint
+ * <p>
+ * That means, when this property is set to 'test',
+ *  - a commit no longer guarantees that the transaction's modification will 
+ *    survive a system crash or
+ * JVM termination 
+ * - the database may not recover successfully upon restart 
+ * - a near full disk at runtime may cause unexpected errors
+ * - database may be in an inconsistent state
+ * 
+ * This program tests for 
+ * 1. setting the derby.system.durability=test is actually not
+ *    doing the syncs by timing inserts
+ * 2. check if a warning message exists in derby.log 
+ * 3. read log.ctrl file and check if the flag is set or not
+ * 4. check if the log.ctrl file flag is not overwritten for the case when 
+ * database booted with derby.system.durability=test set, then shutdown
+ * and database booted without derby.system.durability=test
+ * 
+ * @author Sunitha Kambhampati 
+ * @version 1.0
+ */
+public class TestDurabilityProperty {
+    public static void main(String[] args) {
+        Connection conn = null;
+        Statement s = null;
+        PreparedStatement ps = null;
+        try {
+            report("1. create database with derby.system.durability=test mode");
+            // use the ij utility to read the property file and
+            // make the initial connection.
+            org.apache.derby.tools.ij.getPropertyArg(args);
+            System.setProperty("derby.system.durability","test");
+            conn = org.apache.derby.tools.ij.startJBMS();
+
+            s = conn.createStatement();
+            s.execute("create table t1 (c1 int, c2 int)");
+            s.close();
+
+            // Test 1
+            // this is a quick check incase someone breaks the
+            // derby.system.durability=test
+            long timeTaken = doInsertsInAutoCommit(conn);
+            conn.close();
+
+            int approxUpperBound = 3000; // approximate upper bound in
+                                         // millisecond
+            if (timeTaken > approxUpperBound) {
+                report("FAIL -- derby.system.durability=test mode seems to be broken.");
+                report(" Time to insert rows in test exceeded the usual limit.");
+            }
+
+            String derbyHome = System.getProperty("derby.system.home");
+            // Test 2
+            // Check if derby.log has the warning message
+            report("Is warning message about derby.system.durability=test present in derby.log ="
+                    + isMessageInDerbyLog(derbyHome));
+            // Test 3
+            // Check if marker is correctly written out to database
+            markerInControlFile(derbyHome);
+            
+            // Test 4
+            // shutdown database and boot database afresh without 
+            // derby.system.durability set to test. In this case the derby.log 
+            // and the log control file should still have the marker that this 
+            // mode was once used to boot database.
+            report(
+                "2. shutdown database and reboot database without " +
+                "derby.system.durability=test and test for marker in log.ctrl file");
+            markerNotOverwritten(derbyHome);
+
+        } catch (Throwable e) {
+            report("FAIL -- unexpected exception: " + e);
+            e.printStackTrace();
+        }
+
+    }
+
+    /**
+     * Note doing inserts in autocommit mode is probably the worst case scenario
+     * in terms of performance as each commit will involve a flush/sync to disk
+     * but in case of the derby.system.durability=test mode, the syncs dont 
+     * happen.
+     * This test case times the inserts and assumes that the inserts on any
+     * system will be less than three second for 500 inserts. Note this upper
+     * bound on time is just an approximate estimation
+     */
+    public static long doInsertsInAutoCommit(Connection conn) throws Exception {
+        PreparedStatement ps = conn
+                .prepareStatement("insert into t1 values(?,?)");
+        long count = 0;
+
+        long start = System.currentTimeMillis();
+
+        for (int i = 0; i < 500; i++) {
+            ps.setInt(1, i);
+            ps.setInt(2, i);
+            count += ps.executeUpdate();
+        }
+
+        long end = System.currentTimeMillis();
+        report("num successful inserts = " + count);
+
+        return (end - start);
+    }
+
+    /**
+     * When derby.system.durability is set, a warning message is written out to
+     * derby.log indicating that the property is set and that it does not
+     * guarantee recoverability This test tests if a message is written out to
+     * derby.log or not
+     */
+    public static boolean isMessageInDerbyLog(String derbyHome) throws Exception {
+        BufferedReader reader = null;
+        File derbylog = null;
+        try {
+            derbylog = new File(derbyHome, "derby.log");
+            reader = new BufferedReader(new FileReader(derbylog));
+
+            String line = null;
+            while ((line = reader.readLine()) != null) {
+                if (line.indexOf("derby.system.durability=test") != -1)
+                    return true;
+
+            }
+            return false;
+        } finally {
+            if (reader != null) {
+                reader.close();
+            }
+            derbylog = null;
+        }
+    }
+
+    /**
+     * if database is booted with derby.system.durability=test, 
+     * a marker is written out into log control
+     * file to recognize that the database was previously booted in this mode
+     * Test if the marker byte is set correctly or not. See comments in
+     * org.apache.derby.impl.store.log.LogToFile for IS_DURABILITY_TESTMODE_NO_SYNC_FLAG
+     */
+    public static void markerInControlFile(String derbyHome) throws Exception {
+        RandomAccessFile controlFile = null;
+        try {
+            int testModeNoSyncMarkerPosition = 28;
+            byte testModeNoSyncMarker = 0x2;
+            controlFile = new RandomAccessFile(derbyHome
+                    + "/wombat/log/log.ctrl", "r");
+            controlFile.seek(testModeNoSyncMarkerPosition);
+            report("log.ctrl file has durability testMode no sync marker value = "
+                    + ((controlFile.readByte() & testModeNoSyncMarker) != 0) );
+        } finally {
+            if (controlFile != null)
+                controlFile.close();
+
+        }
+    }
+
+    /**
+     * Test for case when database is booted without derby.system.durability=test
+     * but previously has been booted with the derby.system.durability=test. In 
+     * this scenario,the log control file should still have the marker to say
+     * that this mode was set previously, and derby.log must also have a warning
+     * message
+     * @param derbyHome value of derby.system.home where the database is
+     * @throws Exception
+     */
+    public static void markerNotOverwritten(String derbyHome) throws Exception
+    {
+        // shutdown database
+        Connection conn = null;
+        try
+        {
+            conn = DriverManager.getConnection("jdbc:derby:;shutdown=true");
+        }
+        catch(Exception e)
+        {
+            report("expected exception for shutdown." + e.getMessage());
+        }
+        // unset property
+        System.setProperty("derby.system.durability","");
+        conn = org.apache.derby.tools.ij.startJBMS();
+        conn.close();
+        markerInControlFile(derbyHome);
+        report("Is warning message about derby.system.durability=test present in derby.log ="
+                + isMessageInDerbyLog(derbyHome));
+    }
+    
+    /**
+     * print message
+     * @param msg to print out 
+     */
+    public static void report(String msg) {
+        System.out.println(msg);
+    }
+
+}