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 2012/05/31 20:40:25 UTC

svn commit: r1344858 - in /db/derby/code/branches/10.9: ./ java/engine/org/apache/derby/iapi/sql/dictionary/ java/engine/org/apache/derby/iapi/store/access/ java/engine/org/apache/derby/iapi/store/raw/ java/engine/org/apache/derby/iapi/store/raw/xact/ ...

Author: mikem
Date: Thu May 31 18:40:24 2012
New Revision: 1344858

URL: http://svn.apache.org/viewvc?rev=1344858&view=rev
Log:
DERBY-5494 Same value returned by successive calls to a sequence generator flanking an unorderly shutdown.
DERBY-5780 identity column performance has degredated

backporting change #1344065 from trunk to 10.9 branch.

The previous patch for DERBY-5494 had the unintended affect of forcing a
synchronous write for all nested user transactions at abort time.  This
in turn caused identity column inserts to have one synchronous write per
insert as the nested user transaction is destroyed for each insert which
does an abort each time.

To solve this interfaces were changed so that calling code could set the
default commit sync behavior when the transaction was committed rather than
count on the "type" of transaction.  Nested user transactions used for identity
columns have default set to not sync, and the rest of the nested user 
transactions default to syncing.  Behavior of other types of transactions 
should not be affected.  User transactions still sync by default and internal 
and ntt's still default to not sync.



Modified:
    db/derby/code/branches/10.9/   (props changed)
    db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/DDLConstantAction.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/heap/Heap.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/RawStore.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/InternalXact.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
    db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java
    db/derby/code/branches/10.9/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
    db/derby/code/branches/10.9/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java

Propchange: db/derby/code/branches/10.9/
------------------------------------------------------------------------------
  Merged /db/derby/code/trunk:r1344065

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java Thu May 31 18:40:24 2012
@@ -702,7 +702,10 @@ public class SPSDescriptor extends Tuple
 				TransactionController nestedTC;
 				try
 				{
-					nestedTC = lcc.getTransactionCompile().startNestedUserTransaction(false);
+					nestedTC = 
+                        lcc.getTransactionCompile().startNestedUserTransaction(
+                            false, true);
+
                     // DERBY-3693: The nested transaction may run into a lock
                     // conflict with its parent transaction, in which case we
                     // don't want to wait for a timeout. If a lock timeout is

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/access/TransactionController.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/access/TransactionController.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/access/TransactionController.java Thu May 31 18:40:24 2012
@@ -742,19 +742,30 @@ public interface TransactionController
      *   moving tuples around in heap and indexes.  changed with DERBY-5493 
      *   to do synchronous commit. code in DDLConstantAction.java.
      * o autoincrement/generated key case.  Kept behavior previous to 
-     *   DERBY-5493 by changing to use commitNoSync.  Changing every 
+     *   DERBY-5493 by changing to use commitNoSync, and defaulting 
+     *   flush_log_on_xact_end to false.  Changing every 
      *   key allocation to be a synchronous commit would be a huge performance
      *   problem for existing applications depending on current performance.
      *   code in InsertResultSet.java
      *
-     * @param readOnly  Is transaction readonly?  Only 1 non-readonly nested
-     *                  transaction is allowed per transaction.
+     * @param readOnly                 Is transaction readonly?  Only 1 non-read
+     *                                 only nested transaction is allowed per 
+     *                                 transaction.
+     *
+     * @param flush_log_on_xact_end    By default should the transaction commit
+     *                                 and abort be synced to the log.  Normal
+     *                                 usage should pick true, unless there is
+     *                                 specific performance need and usage 
+     *                                 works correctly if a commit can be lost
+     *                                 on system crash.
      *
 	 * @return The new nested user transaction.
      *
 	 * @exception  StandardException  Standard exception policy.
      **/
-    public TransactionController startNestedUserTransaction(boolean readOnly)
+    public TransactionController startNestedUserTransaction(
+    boolean readOnly,
+    boolean flush_log_on_xact_end)
         throws StandardException;
 
     /**

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java Thu May 31 18:40:24 2012
@@ -723,6 +723,10 @@ public interface RawStoreFactory extends
         thrown if context is not the current context.
         @param transName is the name of the transaction. This name will be 
         displayed by the transactiontable VTI.
+        @param flush_log_on_xact_end    By default should the transaction 
+        commit and abort be synced to the log.  Normal usage should pick true, 
+        unless there is specific performance need and usage works correctly if 
+        a commit can be lost on system crash.
 
         @exception StandardException Standard Derby error policy
 
@@ -733,7 +737,8 @@ public interface RawStoreFactory extends
 
     public Transaction startNestedUpdateUserTransaction(
     ContextManager contextMgr,
-    String         transName)
+    String         transName,
+    boolean        flush_log_on_xact_end)
         throws StandardException;
 
 

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java Thu May 31 18:40:24 2012
@@ -117,10 +117,16 @@ public interface TransactionFactory exte
         will push a transaction context as described in
         RawStoreFactory.startNestedTransaction
 
-        @param contextMgr           is the context manager to use.  It must be 
-                                    the current context manager.
-        @param transName            is the transaction name. It will be 
-                                    displayed in the transactiontable VTI.
+        @param contextMgr               is the context manager to use.  It must
+                                        be the current context manager.
+        @param transName                is the transaction name. It will be 
+                                        displayed in the transactiontable VTI.
+        @param flush_log_on_xact_end    By default should the transaction commit
+                                        and abort be synced to the log.  Normal
+                                        usage should pick true, unless there
+                                        is specific performance need and usage
+                                        works correctly if a commit can be 
+                                        lost on system crash.
 
 		@see RawStoreFactory#startNestedUpdateUserTransaction
 
@@ -129,7 +135,8 @@ public interface TransactionFactory exte
 	public RawTransaction startNestedUpdateUserTransaction(
     RawStoreFactory rsf,
     ContextManager  contextMgr,
-    String          transName)
+    String          transName,
+    boolean         flush_log_on_xact_end)
         throws StandardException;
 
 

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java Thu May 31 18:40:24 2012
@@ -272,7 +272,9 @@ public abstract class SequenceUpdater im
             // contention. Since this is a read-only subtransaction, there should be
             // no conflict with the parent transaction.
             //
-            TransactionController subTransaction = executionTC.startNestedUserTransaction( true );
+            TransactionController subTransaction = 
+                executionTC.startNestedUserTransaction( true, true );
+
             try {
                 _sequenceGenerator = createSequenceGenerator( subTransaction );
             }
@@ -414,7 +416,8 @@ public abstract class SequenceUpdater im
 		}
 
         TransactionController executionTransaction = lcc.getTransactionExecute();
-        TransactionController nestedTransaction = executionTransaction.startNestedUserTransaction( false );
+        TransactionController nestedTransaction = 
+            executionTransaction.startNestedUserTransaction( false, true );
 
         if ( nestedTransaction != null )
         {

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java Thu May 31 18:40:24 2012
@@ -1897,7 +1897,8 @@ public class GenericLanguageConnectionCo
         }
 
         if (readOnlyNestedTransaction == null)
-            readOnlyNestedTransaction = tran.startNestedUserTransaction(readOnly);
+            readOnlyNestedTransaction = 
+                tran.startNestedUserTransaction(readOnly, true);
 
         queryNestingDepth++;
     }

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java Thu May 31 18:40:24 2012
@@ -793,7 +793,7 @@ class AlterTableConstantAction extends D
 		try {
 
             nested_tc = 
-                tc.startNestedUserTransaction(false);
+                tc.startNestedUserTransaction(false, true);
 
             switch (td.getTableType())
             {

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/DDLConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/DDLConstantAction.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/DDLConstantAction.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/DDLConstantAction.java Thu May 31 18:40:24 2012
@@ -164,7 +164,7 @@ abstract class DDLConstantAction impleme
 		TransactionController nestedTc = null;
 
 		try {
-			nestedTc = tc.startNestedUserTransaction(false);
+			nestedTc = tc.startNestedUserTransaction(false, true);
 			useTc = nestedTc;
 		} catch (StandardException e) {
 			if (SanityManager.DEBUG) {

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java Thu May 31 18:40:24 2012
@@ -793,7 +793,15 @@ class InsertResultSet extends DMLWriteRe
 
 			try
 			{
-				nestedTC = tc.startNestedUserTransaction(false);
+                // DERBY-5780, defaulting log syncing to false, which improves
+                // performance of identity value generation.  If system 
+                // crashes may reuse an identity value because commit did not
+                // sync, but only if no subsequent user transaction has 
+                // committed or aborted and thus no row can exist that used
+                // the previous value.  Without this identity values pay
+                // a synchronous I/O to the log file for each new value no
+                // matter how many are inserted in a single transaction.
+				nestedTC = tc.startNestedUserTransaction(false, false);
 				tcToUse = nestedTC;
 			}
 			catch (StandardException se)

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java Thu May 31 18:40:24 2012
@@ -2305,11 +2305,24 @@ public class RAMTransaction 
      * read locks accumulated while compiling a plan, and auto-increment.
      * <p>
      *
+     * @param readOnly                 Is transaction readonly?  Only 1 non-read
+     *                                 only nested transaction is allowed per 
+     *                                 transaction.
+     *
+     * @param flush_log_on_xact_end    By default should the transaction commit
+     *                                 and abort be synced to the log.  Normal
+     *                                 usage should pick true, unless there is
+     *                                 specific performance need and usage 
+     *                                 works correctly if a commit can be lost
+     *                                 on system crash.
+     *
 	 * @return The new nested user transaction.
      *
 	 * @exception  StandardException  Standard exception policy.
      **/
-    public TransactionController startNestedUserTransaction(boolean readOnly)
+    public TransactionController startNestedUserTransaction(
+    boolean readOnly,
+    boolean flush_log_on_xact_end)
         throws StandardException
     {
         // Get the context manager.
@@ -2330,10 +2343,13 @@ public class RAMTransaction 
         Transaction child_rawtran = 
             ((readOnly) ?
                 accessmanager.getRawStore().startNestedReadOnlyUserTransaction(
-                    getLockSpace(), cm,
+                    getLockSpace(), 
+                    cm,
                     AccessFactoryGlobals.NESTED_READONLY_USER_TRANS) :
                 accessmanager.getRawStore().startNestedUpdateUserTransaction(
-                    cm, AccessFactoryGlobals.NESTED_UPDATE_USER_TRANS));
+                    cm, 
+                    AccessFactoryGlobals.NESTED_UPDATE_USER_TRANS,
+                    flush_log_on_xact_end));
 
         RAMTransaction rt   = 
             new RAMTransaction(accessmanager, child_rawtran, this);
@@ -2347,11 +2363,6 @@ public class RAMTransaction 
         child_rawtran.setDefaultLockingPolicy(
                 accessmanager.getDefaultLockingPolicy());
 
-        /*
-        System.out.println("returning nested xact: " + rt + 
-                "child_rawtran = " + child_rawtran); 
-                */
-
         return(rt);
     }
 

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/heap/Heap.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/heap/Heap.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/heap/Heap.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/access/heap/Heap.java Thu May 31 18:40:24 2012
@@ -823,7 +823,7 @@ public class Heap 
             // locks on the table, otherwise the purge will fail with a self
             // deadlock.
             nested_xact = (TransactionManager) 
-                xact_manager.startNestedUserTransaction(false);
+                xact_manager.startNestedUserTransaction(false, true);
 
             // now open the table in a nested user transaction so that each
             // page worth of work can be committed after it is done.

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/RawStore.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/RawStore.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/RawStore.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/RawStore.java Thu May 31 18:40:24 2012
@@ -444,12 +444,13 @@ public final class RawStore implements R
 
 	public Transaction startNestedUpdateUserTransaction(
     ContextManager  contextMgr,
-    String          transName)
+    String          transName,
+    boolean         flush_log_on_xact_end)
         throws StandardException
     {
 		return(
             xactFactory.startNestedUpdateUserTransaction(
-                this, contextMgr, transName));
+                this, contextMgr, transName, flush_log_on_xact_end));
 	}
 
 	public Transaction findUserTransaction(

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/InternalXact.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/InternalXact.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/InternalXact.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/InternalXact.java Thu May 31 18:40:24 2012
@@ -57,7 +57,7 @@ public class InternalXact extends Xact  
     {
 		super(
             xactFactory, logFactory, dataFactory, dataValueFactory, 
-            false, null);
+            false, null, false);
 
 		// always want to hold latches & containers open past the commit/abort
 		setPostComplete();

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java Thu May 31 18:40:24 2012
@@ -243,6 +243,20 @@ public class Xact extends RawTransaction
     // The transaction is only allowed read operations, no log writes.
     private boolean         readOnly;
 
+    // Whether or not to flush log on commit or abort.  
+    // Current usage:
+    // User transactions default to flush.  Internal and nested top
+    // transactions default to not flush.  
+    //
+    // Nested user update transactions are configured when they are created, 
+    // and most default to flush.  Nested user update transaction used for
+    // identity column maintenance defaults to not flush to maintain 
+    // backward performance compatibility with previous releases.
+    //
+    // In all cases log will not be flushsed by Xact.prepareCommit()
+    // if commitNoSync() has been called rather than commit.
+    private boolean         flush_log_on_xact_end;
+
 	// true, if the transaction executed some operations(like unlogged
 	// operations) that block the  online backup to prevent inconsistent
 	// backup copy.
@@ -264,16 +278,18 @@ public class Xact extends RawTransaction
     DataFactory         dataFactory,
     DataValueFactory    dataValueFactory,
     boolean             readOnly,
-    CompatibilitySpace  compatibilitySpace)
+    CompatibilitySpace  compatibilitySpace,
+    boolean             flush_log_on_xact_end)
     {
 
 		super();
 
-		this.xactFactory        = xactFactory;
-		this.logFactory         = logFactory;
-		this.dataFactory        = dataFactory;
-		this.dataValueFactory   = dataValueFactory;
-		this.readOnly           = readOnly;
+		this.xactFactory            = xactFactory;
+		this.logFactory             = logFactory;
+		this.dataFactory            = dataFactory;
+		this.dataValueFactory       = dataValueFactory;
+		this.readOnly               = readOnly;
+		this.flush_log_on_xact_end  = flush_log_on_xact_end;
 
 		if (compatibilitySpace == null) {
 			this.compatibilitySpace =
@@ -298,11 +314,6 @@ public class Xact extends RawTransaction
 		setIdleState();
 
 		backupBlocked = false; 
-
-        /*
-        System.out.println("Xact.constructor: readonly = " + this.readOnly +
-                ";this = " + this);
-                */
 	}
 
 
@@ -773,8 +784,8 @@ public class Xact extends RawTransaction
 
 			// flush the log.
 
-			if (seenUpdates) {
-
+			if (seenUpdates) 
+            {
 				EndXact ex = 
                     new EndXact(
                         getGlobalId(), 
@@ -784,7 +795,7 @@ public class Xact extends RawTransaction
 
 				flushTo = logger.logAndDo(this, ex);
 
-				if (xactFactory.flushLogOnCommit(xc.getIdName()))
+				if (flush_log_on_xact_end)
 				{
 					if ((commitflag & COMMIT_SYNC) == 0)
                     {

Modified: db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java (original)
+++ db/derby/code/branches/10.9/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java Thu May 31 18:40:24 2012
@@ -312,13 +312,14 @@ public class XactFactory implements Tran
 	 * @exception  StandardException  Standard exception policy.
      **/
 	private RawTransaction startCommonTransaction(
-    RawStoreFactory rsf,
-    ContextManager  cm,
-    boolean         readOnly,
-    CompatibilitySpace compatibilitySpace,
-    String          xact_context_id,
-    String          transName,
-    boolean         excludeMe)
+    RawStoreFactory     rsf, 
+    ContextManager      cm,
+    boolean             readOnly,
+    CompatibilitySpace  compatibilitySpace,
+    String              xact_context_id,
+    String              transName,
+    boolean             excludeMe,
+    boolean             flush_log_on_xact_end)
         throws StandardException
     {
 
@@ -335,7 +336,7 @@ public class XactFactory implements Tran
 		Xact xact = 
             new Xact(
                 this, logFactory, dataFactory, dataValueFactory, 
-                readOnly, compatibilitySpace);
+                readOnly, compatibilitySpace, flush_log_on_xact_end);
 
         xact.setTransName(transName);
 		pushTransactionContext(cm, xact_context_id, xact,
@@ -351,8 +352,16 @@ public class XactFactory implements Tran
     String transName)
         throws StandardException
     {
-        return(startCommonTransaction(
-                rsf, cm, false, null, USER_CONTEXT_ID, transName, true));
+        return(
+            startCommonTransaction(
+                rsf, 
+                cm, 
+                false,              // user xact always read/write 
+                null, 
+                USER_CONTEXT_ID, 
+                transName, 
+                true,               // user xact always excluded during quiesce
+                true));             // user xact default flush on xact end
 	}
 
 	public RawTransaction startNestedReadOnlyUserTransaction(
@@ -362,20 +371,39 @@ public class XactFactory implements Tran
     String          transName)
         throws StandardException
     {
-        return(startCommonTransaction(
-            rsf, cm, true, compatibilitySpace, 
-            NESTED_READONLY_USER_CONTEXT_ID, transName, false));
+        return(
+            startCommonTransaction(
+                rsf, 
+                cm, 
+                true, 
+                compatibilitySpace, 
+                NESTED_READONLY_USER_CONTEXT_ID, 
+                transName, 
+                false,
+                true));             // user readonly xact default flush on xact
+                                    // end, should never have anything to flush.
 	}
 
 	public RawTransaction startNestedUpdateUserTransaction(
     RawStoreFactory rsf,
     ContextManager  cm,
-    String          transName)
+    String          transName,
+    boolean         flush_log_on_xact_end)
         throws StandardException
     {
-        return(startCommonTransaction(
-            rsf, cm, false, null, 
-            NESTED_UPDATE_USER_CONTEXT_ID, transName, true));
+        return(
+            startCommonTransaction(
+                rsf, 
+                cm, 
+                false, 
+                null, 
+                NESTED_UPDATE_USER_CONTEXT_ID, 
+                transName, 
+                true,
+                flush_log_on_xact_end));    // allow caller to choose default 
+                                            // log log flushing on commit/abort
+                                            // for internal operations used 
+                                            // nested user update transaction.
 	}
 
 	public RawTransaction startGlobalTransaction(
@@ -395,8 +423,14 @@ public class XactFactory implements Tran
 
         RawTransaction xact = 
             startCommonTransaction(
-                rsf, cm, false, null, 
-                USER_CONTEXT_ID, AccessFactoryGlobals.USER_TRANS_NAME, true);
+                rsf, 
+                cm, 
+                false, 
+                null, 
+                USER_CONTEXT_ID, 
+                AccessFactoryGlobals.USER_TRANS_NAME, 
+                true,
+                true);             // user xact default flush on xact end
 
         xact.setTransactionId(gid, xact.getId());
 
@@ -443,7 +477,8 @@ public class XactFactory implements Tran
 
 		Xact xact = 
             new Xact(
-                this, logFactory, dataFactory, dataValueFactory, false, null);
+                this, logFactory, dataFactory, dataValueFactory, 
+                false, null, false);
 
 		// hold latches etc. past commit in NTT
 		xact.setPostComplete();
@@ -937,28 +972,6 @@ public class XactFactory implements Tran
 	}
 
 	/**
-		Decide if a transaction of this contextId needs to flush the log when
-		it commits
-	*/
-	public boolean flushLogOnCommit(String contextName)
-	{
-		//
-		// if this is a user transaction, flush the log by default.
-        // if this is a nested user update transaction, flush log by default.
-		// if this is an internal or nested top transaction, do not
-		// flush, let it age out.
-        //
-        // In all cases log will not be flushsed by Xact.prepareCommit() 
-        // if commitNoSync() has been called rather than commit.
-		//
-		return (contextName == USER_CONTEXT_ID               || 
-				contextName.equals(USER_CONTEXT_ID)          ||
-                contextName == NESTED_UPDATE_USER_CONTEXT_ID ||
-                contextName.equals(NESTED_UPDATE_USER_CONTEXT_ID));
-	}
-
-
-	/**
 		Get a locking policy for a transaction.
 	*/
 	final LockingPolicy getLockingPolicy(

Modified: db/derby/code/branches/10.9/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java (original)
+++ db/derby/code/branches/10.9/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java Thu May 31 18:40:24 2012
@@ -118,7 +118,9 @@ class NoOpTransaction implements Transac
         return 0;
     }
 
-    public TransactionController startNestedUserTransaction(boolean readOnly)
+    public TransactionController startNestedUserTransaction(
+    boolean readOnly,
+    boolean flush_log_on_xact_end)
             throws StandardException {
         return this;
     }

Modified: db/derby/code/branches/10.9/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.9/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java?rev=1344858&r1=1344857&r2=1344858&view=diff
==============================================================================
--- db/derby/code/branches/10.9/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java (original)
+++ db/derby/code/branches/10.9/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java Thu May 31 18:40:24 2012
@@ -2987,7 +2987,8 @@ public class T_AccessFactory extends T_G
                 ContextService.getFactory().getCurrentContextManager());
 
         // get a nested user transaction
-        TransactionController child_tc = tc.startNestedUserTransaction(true);
+        TransactionController child_tc = 
+            tc.startNestedUserTransaction(true, true);
 
         TransactionController current_xact_after_nest = 
             store.getTransaction(
@@ -3033,11 +3034,11 @@ public class T_AccessFactory extends T_G
         tc.dropConglomerate(orig_conglomid);
 
         // trying to double nest a nested transaction should not work.
-        child_tc = tc.startNestedUserTransaction(true);
+        child_tc = tc.startNestedUserTransaction(true, true);
 
         try
         {
-            child_tc.startNestedUserTransaction(true);
+            child_tc.startNestedUserTransaction(true, true);
 
 			throw T_Fail.testFailMsg(
                 "(nestedUserTransaction) double nest xact not allowed.");
@@ -3074,7 +3075,7 @@ public class T_AccessFactory extends T_G
                 null,         // default properties
                 TransactionController.IS_DEFAULT);       // not temporary
 
-        child_tc = tc.startNestedUserTransaction(true);
+        child_tc = tc.startNestedUserTransaction(true, true);
 
         // add 20 pages worth of data, causing allocation
         
@@ -3132,7 +3133,7 @@ public class T_AccessFactory extends T_G
         tc.commit();
 
 
-        child_tc = tc.startNestedUserTransaction(true);
+        child_tc = tc.startNestedUserTransaction(true, true);
 
         // add 20 pages worth of data, causing allocation
         
@@ -3183,7 +3184,7 @@ public class T_AccessFactory extends T_G
         }
 
         // start an read only nested user transaction.
-        child_tc = tc.startNestedUserTransaction(true);
+        child_tc = tc.startNestedUserTransaction(true, true);
 
         ConglomerateController child_cc = 
             child_tc.openConglomerate(
@@ -3213,7 +3214,7 @@ public class T_AccessFactory extends T_G
         tc.commit();
 
         // start an update nested user transaction.
-        child_tc = tc.startNestedUserTransaction(false);
+        child_tc = tc.startNestedUserTransaction(false, true);
 
         child_cc = 
             child_tc.openConglomerate(
@@ -3253,7 +3254,7 @@ public class T_AccessFactory extends T_G
                 TransactionController.ISOLATION_SERIALIZABLE);
 
         // start an update nested user transaction.
-        child_tc = tc.startNestedUserTransaction(false);
+        child_tc = tc.startNestedUserTransaction(false, true);
 
         try 
         {