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 ka...@apache.org on 2007/05/07 13:45:33 UTC

svn commit: r535860 - in /db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute: BulkTableScanResultSet.java HashScanResultSet.java LastIndexKeyResultSet.java ScanResultSet.java TableScanResultSet.java

Author: kahatlen
Date: Mon May  7 04:45:32 2007
New Revision: 535860

URL: http://svn.apache.org/viewvc?view=rev&rev=535860
Log:
DERBY-2597: Language result sets should not reuse current isolation
level across executions

Let HashScanResultSet, TableScanResultSet and LastIndexKeyResultSet
refresh the isolation level on each open.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BulkTableScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/LastIndexKeyResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TableScanResultSet.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BulkTableScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BulkTableScanResultSet.java?view=diff&rev=535860&r1=535859&r2=535860
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BulkTableScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BulkTableScanResultSet.java Mon May  7 04:45:32 2007
@@ -358,7 +358,7 @@
 	 */
 	protected boolean canGetInstantaneousLocks()
 	{
-		return true;
+		return !forUpdate;
 	}
 
 	/**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java?view=diff&rev=535860&r1=535859&r2=535860
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/HashScanResultSet.java Mon May  7 04:45:32 2007
@@ -67,7 +67,7 @@
  * collision occurs, the store builds a <code>List</code> with the colliding
  * <code>DataValueDescriptor[]</code>s.
  */
-public class HashScanResultSet extends NoPutResultSetImpl
+public class HashScanResultSet extends ScanResultSet
 	implements CursorResultSet
 {
 	private boolean		hashtableBuilt;
@@ -102,8 +102,6 @@
 	public boolean forUpdate;
 	private boolean runTimeStatisticsOn;
 	private FormatableBitSet accessedCols;
-	public int isolationLevel;
-	public int lockMode;
 	public int[] keyColumns;
 	private boolean sameStartStopPosition;
 	private boolean skipNullKeyColumns;
@@ -155,6 +153,7 @@
     {
 		super(activation,
 				resultSetNumber,
+				lockMode, tableLocked, isolationLevel,
 				optimizerEstimatedRowCount,
 				optimizerEstimatedCost);
         this.scoci = scoci;
@@ -208,69 +207,6 @@
 			this.accessedCols = (FormatableBitSet)(activation.getPreparedStatement().
 										  getSavedObject(colRefItem));
 		}
-		this.lockMode = lockMode;
-
-		/* Isolation level - translate from language to store */
-		// If not specified, get current isolation level
-		if (isolationLevel == ExecutionContext.UNSPECIFIED_ISOLATION_LEVEL)
-		{
-			isolationLevel = lcc.getCurrentIsolationLevel();
-		}
-
-        if (isolationLevel == ExecutionContext.SERIALIZABLE_ISOLATION_LEVEL)
-        {
-            this.isolationLevel = TransactionController.ISOLATION_SERIALIZABLE;
-        }
-        else
-        {
-            /* NOTE: always do row locking on READ COMMITTED/UNCOMMITTED 
-             *       and repeatable read scans unless the table is marked as 
-             *       table locked (in sys.systables).
-             *
-             *		 We always get instantaneous locks as we will complete
-             *		 the scan before returning any rows and we will fully
-             *		 requalify the row if we need to go to the heap on a next().
-             */
-
-            if (! tableLocked)
-            {
-                this.lockMode = TransactionController.MODE_RECORD;
-            }
-
-            if (isolationLevel == 
-                    ExecutionContext.READ_COMMITTED_ISOLATION_LEVEL)
-            {
-                this.isolationLevel = 
-                    TransactionController.ISOLATION_READ_COMMITTED_NOHOLDLOCK;
-            }
-            else if (isolationLevel == 
-                        ExecutionContext.READ_UNCOMMITTED_ISOLATION_LEVEL)
-            {
-                this.isolationLevel = 
-                    TransactionController.ISOLATION_READ_UNCOMMITTED;
-            }
-            else if (isolationLevel == 
-                        ExecutionContext.REPEATABLE_READ_ISOLATION_LEVEL)
-            {
-                this.isolationLevel = 
-                    TransactionController.ISOLATION_REPEATABLE_READ;
-            }
-        }
-
-        if (SanityManager.DEBUG)
-        {
-            SanityManager.ASSERT(
-                ((isolationLevel == 
-                      ExecutionContext.READ_COMMITTED_ISOLATION_LEVEL)   ||
-                 (isolationLevel == 
-                      ExecutionContext.READ_UNCOMMITTED_ISOLATION_LEVEL) ||
-                 (isolationLevel == 
-                      ExecutionContext.REPEATABLE_READ_ISOLATION_LEVEL)  ||
-                 (isolationLevel == 
-                      ExecutionContext.SERIALIZABLE_ISOLATION_LEVEL)),
-
-                "Invalid isolation level - " + isolationLevel);
-        }
 
 		runTimeStatisticsOn = 
             getLanguageConnectionContext().getRunTimeStatisticsMode();
@@ -287,6 +223,14 @@
 	//
 
 	/**
+	 * Can we get instantaneous locks when getting share row
+	 * locks at READ COMMITTED.
+	 */
+	boolean canGetInstantaneousLocks() {
+		return true;
+	}
+
+	/**
      * open a scan on the table. scan parameters are evaluated
      * at each open, so there is probably some way of altering
      * their values...
@@ -304,6 +248,8 @@
         // Get the current transaction controller
         tc = activation.getTransactionController();
 
+		initIsolationLevel();
+
 		if (startKeyGetter != null)
 		{
 			startPosition = (ExecIndexRow) startKeyGetter.invoke(activation);
@@ -652,14 +598,6 @@
 		{
 			return totTime;
 		}
-	}
-
-	/**
-	 * @see NoPutResultSet#getScanIsolationLevel
-	 */
-	public int getScanIsolationLevel()
-	{
-		return isolationLevel;
 	}
 
 	/**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/LastIndexKeyResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/LastIndexKeyResultSet.java?view=diff&rev=535860&r1=535859&r2=535860
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/LastIndexKeyResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/LastIndexKeyResultSet.java Mon May  7 04:45:32 2007
@@ -55,7 +55,7 @@
  * max().
  *
  */
-class LastIndexKeyResultSet extends NoPutResultSetImpl
+class LastIndexKeyResultSet extends ScanResultSet
 {
 	protected	ExecRow		candidate;
 
@@ -74,9 +74,6 @@
 	protected boolean runTimeStatisticsOn;
 	protected FormatableBitSet accessedCols;
 
-	public int isolationLevel;
-	public int lockMode;
-
 	// Run time statistics
 	public String stopPositionString;
 	public boolean coarserLock;
@@ -131,6 +128,7 @@
 	{
 		super(activation,
 				resultSetNumber,
+				lockMode, tableLocked, isolationLevel,
 				optimizerEstimatedRowCount,
 				optimizerEstimatedCost);
 
@@ -146,73 +144,11 @@
 		this.tableName = tableName;
 		this.userSuppliedOptimizerOverrides = userSuppliedOptimizerOverrides;
 		this.indexName = indexName;
-		this.lockMode = lockMode;
 		if (colRefItem != -1)
 		{
 			this.accessedCols = (FormatableBitSet)(activation.getPreparedStatement().
 						getSavedObject(colRefItem));
 		}
-		/* Isolation level - translate from language to store */
-		// If not specified, get current isolation level
-		if (isolationLevel == ExecutionContext.UNSPECIFIED_ISOLATION_LEVEL)
-		{
-			isolationLevel = lcc.getCurrentIsolationLevel();
-		}
-
-        if (isolationLevel == ExecutionContext.SERIALIZABLE_ISOLATION_LEVEL)
-        {
-            this.isolationLevel = TransactionController.ISOLATION_SERIALIZABLE;
-        }
-        else
-        {
-            /* NOTE: always do row locking on READ COMMITTED/UNCOMMITTED 
-             *       and repeatable read scans unless the table is marked as 
-             *       table locked (in sys.systables).
-             *
-             *		 We always get instantaneous locks as we will complete
-             *		 the scan before returning any rows and we will fully
-             *		 requalify the row if we need to go to the heap on a next().
-             */
-
-            if (! tableLocked)
-            {
-                this.lockMode = TransactionController.MODE_RECORD;
-            }
-
-            if (isolationLevel == 
-                    ExecutionContext.READ_COMMITTED_ISOLATION_LEVEL)
-            {
-                this.isolationLevel = 
-                    TransactionController.ISOLATION_READ_COMMITTED_NOHOLDLOCK;
-            }
-            else if (isolationLevel == 
-                        ExecutionContext.READ_UNCOMMITTED_ISOLATION_LEVEL)
-            {
-                this.isolationLevel = 
-                    TransactionController.ISOLATION_READ_UNCOMMITTED;
-            }
-            else if (isolationLevel == 
-                        ExecutionContext.REPEATABLE_READ_ISOLATION_LEVEL)
-            {
-                this.isolationLevel = 
-                    TransactionController.ISOLATION_REPEATABLE_READ;
-            }
-        }
-
-        if (SanityManager.DEBUG)
-        {
-            SanityManager.ASSERT(
-                ((isolationLevel == 
-                      ExecutionContext.READ_COMMITTED_ISOLATION_LEVEL)   ||
-                 (isolationLevel == 
-                      ExecutionContext.READ_UNCOMMITTED_ISOLATION_LEVEL) ||
-                 (isolationLevel == 
-                      ExecutionContext.REPEATABLE_READ_ISOLATION_LEVEL)  ||
-                 (isolationLevel == 
-                      ExecutionContext.SERIALIZABLE_ISOLATION_LEVEL)),
-
-                "Invalid isolation level - " + isolationLevel);
-        }
 
 		runTimeStatisticsOn = getLanguageConnectionContext().getRunTimeStatisticsMode();
 
@@ -242,6 +178,14 @@
 	/////////////////////////////////////////////////////
 
 	/**
+	 * Can we get instantaneous locks when getting share row
+	 * locks at READ COMMITTED.
+	 */
+	boolean canGetInstantaneousLocks() {
+		return true;
+	}
+
+	/**
 	* open a scan on the table. scan parameters are evaluated
 	* at each open, so there is probably some way of altering
 	* their values...
@@ -260,6 +204,8 @@
 
 		isOpen = true;
 		TransactionController tc = activation.getTransactionController();
+
+		initIsolationLevel();
 
 		/*
 		** Grab the last row.  Note that if there are deletes

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java?view=auto&rev=535860
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java Mon May  7 04:45:32 2007
@@ -0,0 +1,189 @@
+/*
+ * Derby - Class org.apache.derby.impl.sql.execute.ScanResultSet
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you 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.derby.impl.sql.execute;
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.sql.Activation;
+import org.apache.derby.iapi.sql.execute.ExecutionContext;
+import org.apache.derby.iapi.store.access.TransactionController;
+import org.apache.derby.shared.common.sanity.SanityManager;
+
+/**
+ * Abstract <code>ResultSet</code> class for <code>NoPutResultSet</code>s which
+ * contain a scan.
+ */
+abstract class ScanResultSet extends NoPutResultSetImpl {
+
+    /** If true, the table is marked as table locked in SYS.SYSTABLES. */
+    private final boolean tableLocked;
+    /** If true, the isolation level is unspecified and must be refreshed on
+     * each open. */
+    private final boolean unspecifiedIsolationLevel;
+    /** The lock mode supplied through the constructor. */
+    private final int suppliedLockMode;
+    /** Tells whether the isolation level needs to be updated. */
+    private boolean isolationLevelNeedsUpdate;
+
+    /** The actual lock mode used. */
+    int lockMode;
+    /** The scan isolation level. */
+    int isolationLevel;
+
+    /**
+     * Construct a <code>ScanResultSet</code>.
+     *
+     * @param activation the activation
+     * @param resultSetNumber number of the result set (unique within statement)
+     * @param lockMode lock mode (record or table)
+     * @param tableLocked true if marked as table locked in SYS.SYSTABLES
+     * @param isolationLevel language isolation level for the result set
+     * @param optimizerEstimatedRowCount estimated row count
+     * @param optimizerEstimatedCost estimated cost
+     */
+    ScanResultSet(Activation activation, int resultSetNumber,
+                  int lockMode, boolean tableLocked, int isolationLevel,
+                  double optimizerEstimatedRowCount,
+                  double optimizerEstimatedCost) {
+        super(activation, resultSetNumber,
+              optimizerEstimatedRowCount,
+              optimizerEstimatedCost);
+
+        this.tableLocked = tableLocked;
+        suppliedLockMode = lockMode;
+
+        if (isolationLevel == ExecutionContext.UNSPECIFIED_ISOLATION_LEVEL) {
+            unspecifiedIsolationLevel = true;
+            isolationLevel = lcc.getCurrentIsolationLevel();
+        } else {
+            unspecifiedIsolationLevel = false;
+        }
+
+        this.lockMode = getLockMode(isolationLevel);
+        this.isolationLevel =
+            translateLanguageIsolationLevel(isolationLevel);
+    }
+
+    /**
+     * Initialize the isolation level and the lock mode. If the result set was
+     * constructed with an explicit isolation level, or if the isolation level
+     * has already been initialized, this is a no-op. All sub-classes should
+     * invoke this method from their <code>openCore()</code> methods.
+     */
+    void initIsolationLevel() {
+        if (isolationLevelNeedsUpdate) {
+            int languageLevel = lcc.getCurrentIsolationLevel();
+            lockMode = getLockMode(languageLevel);
+            isolationLevel = translateLanguageIsolationLevel(languageLevel);
+            isolationLevelNeedsUpdate = false;
+        }
+    }
+
+    /**
+     * Get the lock mode based on the language isolation level. Always do row
+     * locking unless the isolation level is serializable or the table is
+     * marked as table locked.
+     *
+     * @param languageLevel the (language) isolation level
+     * @return lock mode
+     */
+    private int getLockMode(int languageLevel) {
+        /* NOTE: always do row locking on READ COMMITTED/UNCOMITTED scans,
+         * unless the table is marked as table locked (in sys.systables)
+         * This is to improve concurrency.  Also see FromBaseTable's
+         * updateTargetLockMode (KEEP THESE TWO PLACES CONSISTENT!
+         * bug 4318).
+         */
+        /* NOTE: always do row locking on READ COMMITTED/UNCOMMITTED
+         *       and repeatable read scans unless the table is marked as
+         *       table locked (in sys.systables).
+         *
+         *       We always get instantaneous locks as we will complete
+         *       the scan before returning any rows and we will fully
+         *       requalify the row if we need to go to the heap on a next().
+         */
+        if (tableLocked ||
+                (languageLevel ==
+                     ExecutionContext.SERIALIZABLE_ISOLATION_LEVEL)) {
+            return suppliedLockMode;
+        } else {
+            return TransactionController.MODE_RECORD;
+        }
+    }
+
+    /**
+     * Translate isolation level from language to store.
+     *
+     * @param languageLevel language isolation level
+     * @return store isolation level
+     */
+    private int translateLanguageIsolationLevel(int languageLevel) {
+
+        switch (languageLevel) {
+        case ExecutionContext.READ_UNCOMMITTED_ISOLATION_LEVEL:
+            return TransactionController.ISOLATION_READ_UNCOMMITTED;
+        case ExecutionContext.READ_COMMITTED_ISOLATION_LEVEL:
+            /*
+             * Now we see if we can get instantaneous locks
+             * if we are getting share locks.
+             * (For example, we can get instantaneous locks
+             * when doing a bulk fetch.)
+             */
+            if (!canGetInstantaneousLocks()) {
+                return TransactionController.ISOLATION_READ_COMMITTED;
+            }
+            return TransactionController.ISOLATION_READ_COMMITTED_NOHOLDLOCK;
+        case ExecutionContext.REPEATABLE_READ_ISOLATION_LEVEL:
+            return TransactionController.ISOLATION_REPEATABLE_READ;
+        case ExecutionContext.SERIALIZABLE_ISOLATION_LEVEL:
+            return TransactionController.ISOLATION_SERIALIZABLE;
+        }
+
+        if (SanityManager.DEBUG) {
+            SanityManager.THROWASSERT("Unknown isolation level - " +
+                                      languageLevel);
+        }
+
+        return 0;
+    }
+
+    /**
+     * Can we get instantaneous locks when getting share row
+     * locks at READ COMMITTED.
+     */
+    abstract boolean canGetInstantaneousLocks();
+
+    /**
+     * Return the isolation level of the scan in the result set.
+     */
+    public int getScanIsolationLevel() {
+        return isolationLevel;
+    }
+
+    /**
+     * Close the result set.
+     *
+     * @exception StandardException if an error occurs
+     */
+    public void close() throws StandardException {
+        // need to update isolation level on next open if it was unspecified
+        isolationLevelNeedsUpdate = unspecifiedIsolationLevel;
+        super.close();
+    }
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TableScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TableScanResultSet.java?view=diff&rev=535860&r1=535859&r2=535860
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TableScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TableScanResultSet.java Mon May  7 04:45:32 2007
@@ -71,7 +71,7 @@
  * improve performance.
  *
  */
-class TableScanResultSet extends NoPutResultSetImpl
+class TableScanResultSet extends ScanResultSet
 	implements CursorResultSet, Cloneable
 {
     protected ScanController scanController;
@@ -105,9 +105,6 @@
 	private boolean nextDone;
 	private RowLocation rlTemplate;
 
-	public int isolationLevel;
-	public int lockMode;
-
 	// Run time statistics
 	private Properties scanProperties;
 	public String startPositionString;
@@ -182,6 +179,7 @@
     {
 		super(activation,
 				resultSetNumber,
+				lockMode, tableLocked, isolationLevel,
 				optimizerEstimatedRowCount,
 				optimizerEstimatedCost);
 
@@ -236,91 +234,6 @@
 		if (indexCols != null)
 			activation.setForUpdateIndexScan(this);
 
-		this.lockMode = lockMode;
-
-		/* Isolation level - translate from language to store */
-		// If not specified, get current isolation level
-		if (isolationLevel == ExecutionContext.UNSPECIFIED_ISOLATION_LEVEL)
-		{
-			isolationLevel = lcc.getCurrentIsolationLevel();
-		}
-
-        if (isolationLevel == ExecutionContext.SERIALIZABLE_ISOLATION_LEVEL)
-        {
-            this.isolationLevel = TransactionController.ISOLATION_SERIALIZABLE;
-        }
-        else
-        {
-			/* NOTE: always do row locking on READ COMMITTED/UNCOMITTED scans,
-			 * unless the table is marked as table locked (in sys.systables)
-			 * This is to improve concurrency.  Also see FromBaseTable's
-			 * updateTargetLockMode (KEEP THESE TWO PLACES CONSISTENT!
-			 * bug 4318).
-			 */
-
-            /* NOTE: always do row locking on READ COMMITTED/UNCOMMITTED 
-             *       and repeatable read scans unless the table is marked as 
-             *       table locked (in sys.systables).
-             *
-             *		 We always get instantaneous locks as we will complete
-             *		 the scan before returning any rows and we will fully
-             *		 requalify the row if we need to go to the heap on a next().
-             */
-
-            if (! tableLocked)
-            {
-                this.lockMode = TransactionController.MODE_RECORD;
-            }
-
-            if (isolationLevel == 
-                    ExecutionContext.READ_COMMITTED_ISOLATION_LEVEL)
-            {
-                /* 
-                 * Now we see if we can get instantaneous locks
-                 * if we are getting share locks.
-                 * (For example, we can get instantaneous locks
-                 * when doing a bulk fetch.)
-                 */
-                if ((! forUpdate) && canGetInstantaneousLocks())
-                {
-                    this.isolationLevel = 
-                        TransactionController.ISOLATION_READ_COMMITTED_NOHOLDLOCK;
-                }
-                else
-                {
-                    this.isolationLevel = 
-                        TransactionController.ISOLATION_READ_COMMITTED;
-                }
-            }
-            else if (isolationLevel == 
-                        ExecutionContext.READ_UNCOMMITTED_ISOLATION_LEVEL)
-            {
-                this.isolationLevel = 
-                    TransactionController.ISOLATION_READ_UNCOMMITTED;
-            }
-            else if (isolationLevel == 
-                        ExecutionContext.REPEATABLE_READ_ISOLATION_LEVEL)
-            {
-                this.isolationLevel = 
-                    TransactionController.ISOLATION_REPEATABLE_READ;
-            }
-        }
-
-        if (SanityManager.DEBUG)
-        {
-            SanityManager.ASSERT(
-                ((isolationLevel == 
-                      ExecutionContext.READ_COMMITTED_ISOLATION_LEVEL)   ||
-                 (isolationLevel == 
-                      ExecutionContext.READ_UNCOMMITTED_ISOLATION_LEVEL) ||
-                 (isolationLevel == 
-                      ExecutionContext.REPEATABLE_READ_ISOLATION_LEVEL)  ||
-                 (isolationLevel == 
-                      ExecutionContext.SERIALIZABLE_ISOLATION_LEVEL)),
-
-                "Invalid isolation level - " + isolationLevel);
-        }
-
 		runTimeStatisticsOn = (activation != null &&
 							   activation.getLanguageConnectionContext().getRunTimeStatisticsMode());
 
@@ -353,6 +266,8 @@
         // Get the current transaction controller
         TransactionController tc = activation.getTransactionController();
 
+		initIsolationLevel();
+
 		if (dcoci == null)
 			dcoci = tc.getDynamicCompiledConglomInfo(conglomId);
 
@@ -1442,14 +1357,6 @@
 		}
 
 		return scanProperties;
-	}
-
-	/**
-	 * @see NoPutResultSet#getScanIsolationLevel
-	 */
-	public int getScanIsolationLevel()
-	{
-		return isolationLevel;
 	}
 
 	/**