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 2012/12/07 14:08:12 UTC

svn commit: r1418297 - in /db/derby/code/trunk/java/engine/org/apache/derby: iapi/sql/compile/ iapi/sql/execute/ iapi/types/ impl/sql/compile/ impl/sql/execute/

Author: kahatlen
Date: Fri Dec  7 13:08:04 2012
New Revision: 1418297

URL: http://svn.apache.org/viewvc?rev=1418297&view=rev
Log:
DERBY-6003: Create row templates outside of the generated code

Create and store an ExecRowBuilder in the GenericPreparedStatement
instead of creating a generated method for allocating row templates.

This patch makes ScanResultSet and its sub-classes use the new approach.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ExecRowBuilder.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/DataTypeDescriptor.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseJoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java
    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/DeleteConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DependentResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.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/InsertConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.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/MultiProbeTableScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TableScanResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdatableVTIConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WriteCursorConstantAction.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/compile/JoinStrategy.java Fri Dec  7 13:08:04 2012
@@ -208,7 +208,7 @@ public interface JoinStrategy {
 	 * @param acb	The expression class builder for the activation class
 	 *				we're building
 	 * @param bulkFetch	The amount of bulk fetch to do
-	 * @param resultRowAllocator	A completed method to allocate the result row
+     * @param resultRowTemplate The saved object index of a result row template
 	 * @param colRefItem	The item number of the column reference bit map
 	 * @param lockMode		The lock mode to use when scanning the table
 	 *						(see TransactionController).
@@ -231,7 +231,7 @@ public interface JoinStrategy {
 							OptimizablePredicateList nonStoreRestrictionList,
 							ExpressionClassBuilderInterface acb,
 							int bulkFetch,
-							MethodBuilder resultRowAllocator,
+							int resultRowTemplate,
 							int colRefItem,
 							int indexColItem,
 							int lockMode,

Added: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ExecRowBuilder.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ExecRowBuilder.java?rev=1418297&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ExecRowBuilder.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ExecRowBuilder.java Fri Dec  7 13:08:04 2012
@@ -0,0 +1,134 @@
+/*
+
+   Derby - Class org.apache.derby.iapi.sql.execute.ExecRowBuilder
+
+   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.iapi.sql.execute;
+
+import java.io.Serializable;
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.services.sanity.SanityManager;
+import org.apache.derby.iapi.types.DataTypeDescriptor;
+import org.apache.derby.iapi.types.DataValueDescriptor;
+
+/**
+ * A class used for storing information on how to build {@code ExecRow}
+ * instances. Typically created by the compiler and used during execution
+ * to produce and reset row templates.
+ */
+public class ExecRowBuilder implements Serializable {
+
+    /**
+     * Serial version produced by the serialver utility. Needed in order to
+     * make serialization work reliably across different compilers.
+     */
+    private static final long serialVersionUID = -1078823466492523202L;
+
+    /** If true, the row should be an {@code ExecIndexRow}. */
+    private final boolean indexable;
+
+    /**
+     * Array of templates used for creating NULL values to put in the row.
+     * The templates are either {@code DataValueDescriptor}s or
+     * {@code DataTypeDescriptor}s.
+     */
+    private final Object[] template;
+
+    /** Array of 1-based column numbers for the columns to access. */
+    private final int[] columns;
+
+    /** The number of columns to set in the row. */
+    private int count;
+
+    /** The highest column number in the row. */
+    private int maxColumnNumber;
+
+    /**
+     * Create an instance that produces an {@code ExecRow} instance of
+     * the specified size.
+     *
+     * @param size the number of columns to initialize in the produced row
+     * @param indexable {@code true} if the returned row should be an
+     * {@code ExecIndexRow}, {@code false} otherwise
+     */
+    public ExecRowBuilder(int size, boolean indexable) {
+        this.template = new Object[size];
+        this.columns = new int[size];
+        this.indexable = indexable;
+    }
+
+    /**
+     * Add a template from which a NULL value of the correct type can be
+     * created. It should either be a {@code DataValueDescriptor} or a
+     * {@code DataTypeDescriptor}.
+     *
+     * @param column the column number
+     * @param columnTemplate a template from which a NULL value can be created
+     * (either a {@code DataValueDescriptor} or a {@code DataTypeDescriptor})
+     */
+    public void setColumn(int column, Object columnTemplate) {
+        if (SanityManager.DEBUG &&
+                !(columnTemplate instanceof DataTypeDescriptor) &&
+                !(columnTemplate instanceof DataValueDescriptor)) {
+            SanityManager.THROWASSERT(
+                "Expected DataTypeDescriptor or DataValueDescriptor. Got: " +
+                ((columnTemplate == null) ? columnTemplate :
+                    columnTemplate.getClass().getName()));
+        }
+        template[count] = columnTemplate;
+        columns[count] = column;
+        count++;
+        maxColumnNumber = Math.max(maxColumnNumber, column);
+    }
+
+    /**
+     * Build a new {@code ExecRow} instance with the columns specified by
+     * the {@link #setColumn(int, Object)} method initialized to empty (NULL)
+     * values.
+     *
+     * @param ef an execution factory used to create a row
+     * @return a row initialized with NULL values of the requested types
+     */
+    public ExecRow build(ExecutionFactory ef) throws StandardException {
+        ExecRow row = indexable ?
+                ef.getIndexableRow(maxColumnNumber) :
+                ef.getValueRow(maxColumnNumber);
+
+        for (int i = 0; i < count; i++) {
+            Object o = template[i];
+            DataValueDescriptor dvd = (o instanceof DataValueDescriptor) ?
+                    ((DataValueDescriptor) o).getNewNull() :
+                    ((DataTypeDescriptor) o).getNull();
+            row.setColumn(columns[i], dvd);
+        }
+
+        return row;
+    }
+
+    /**
+     * Reset a row by creating fresh NULL values.
+     * @param row the row to reset
+     */
+    public void reset(ExecRow row) throws StandardException {
+        for (int i = 0; i < count; i++) {
+            int col = columns[i];
+            row.setColumn(col, row.getColumn(col).getNewNull());
+        }
+    }
+}

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

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/execute/ResultSetFactory.java Fri Dec  7 13:08:04 2012
@@ -113,11 +113,13 @@ public interface ResultSetFactory {
 			be inserted into the target table.
 		@param generationClauses	The code to compute column generation clauses if any
 		@param checkGM	The code to enforce the check constraints, if any
+        @param fullTemplate Saved item for a row template used by bulk insert,
+                            or -1 if this is not a bulk insert
 		@return the insert operation as a result set.
 		@exception StandardException thrown when unable to perform the insert
 	 */
 	ResultSet getInsertResultSet(NoPutResultSet source, GeneratedMethod generationClauses,
-								 GeneratedMethod checkGM)
+                                 GeneratedMethod checkGM, int fullTemplate)
         throws StandardException;
 
 	/**
@@ -688,11 +690,7 @@ public interface ResultSetFactory {
 			which provides the context for the row allocation operation.
 		@param conglomId the conglomerate of the table to be scanned.
 		@param scociItem The saved item for the static conglomerate info.
-		@param resultRowAllocator a reference to a method in the activation
-			that creates a holder for the rows from the scan.
-			<verbatim>
-				ExecRow rowAllocator() throws StandardException;
-			</verbatim>
+        @param resultRowTemplate The saved item for result row template.
 		@param resultSetNumber	The resultSetNumber for the ResultSet
 		@param startKeyGetter a reference to a method in the activation
 			that gets the start key indexable row for the scan.  Null
@@ -743,7 +741,7 @@ public interface ResultSetFactory {
 			                    Activation activation,
 								long conglomId,
 								int scociItem,						
-								GeneratedMethod resultRowAllocator,
+								int resultRowTemplate,
 								int resultSetNumber,
 								GeneratedMethod startKeyGetter,
 								int startSearchOperator,
@@ -779,11 +777,7 @@ public interface ResultSetFactory {
 			which provides the context for the row allocation operation.
 		@param conglomId the conglomerate of the table to be scanned.
 		@param scociItem The saved item for the static conglomerate info.
-		@param resultRowAllocator a reference to a method in the activation
-			that creates a holder for the rows from the scan.
-			<verbatim>
-				ExecRow rowAllocator() throws StandardException;
-			</verbatim>
+        @param resultRowTemplate The saved item for result row template.
 		@param resultSetNumber	The resultSetNumber for the ResultSet
 		@param hashKeyColumn	The 0-based column # for the hash key.
 		@param tableName		The full name of the table
@@ -809,7 +803,7 @@ public interface ResultSetFactory {
 			                    Activation activation,
 								long conglomId,
 								int scociItem,			
-								GeneratedMethod resultRowAllocator,
+								int resultRowTemplate,
 								int resultSetNumber,
 								int hashKeyColumn,
 								String tableName,
@@ -838,12 +832,7 @@ public interface ResultSetFactory {
 		@param scociItem The saved item for the static conglomerate info.
 		@param activation the activation for this result set,
 			which provides the context for the row allocation operation.
-		@param resultRowAllocator a reference to a method in the activation
-			that creates a holder for the result row of the scan.  May
-			be a partial row.
-			<verbatim>
-				ExecRow rowAllocator() throws StandardException;
-			</verbatim>
+        @param resultRowTemplate The saved item for result row template.
 		@param resultSetNumber	The resultSetNumber for the ResultSet
 		@param startKeyGetter a reference to a method in the activation
 			that gets the start key indexable row for the scan.  Null
@@ -891,7 +880,7 @@ public interface ResultSetFactory {
 			                    Activation activation,
 								long conglomId,
 								int scociItem,
-								GeneratedMethod resultRowAllocator,
+								int resultRowTemplate,
 								int resultSetNumber,
 								GeneratedMethod startKeyGetter,
 								int startSearchOperator,
@@ -928,12 +917,7 @@ public interface ResultSetFactory {
 		@param scociItem The saved item for the static conglomerate info.
 		@param activation the activation for this result set,
 			which provides the context for the row allocation operation.
-		@param resultRowAllocator a reference to a method in the activation
-			that creates a holder for the result row of the scan.  May
-			be a partial row.
-			<verbatim>
-				ExecRow rowAllocator() throws StandardException;
-			</verbatim>
+        @param resultRowTemplate The saved item for result row template.
 		@param resultSetNumber	The resultSetNumber for the ResultSet
 		@param startKeyGetter a reference to a method in the activation
 			that gets the start key indexable row for the scan.  Null
@@ -984,7 +968,7 @@ public interface ResultSetFactory {
 			                    Activation activation,
 								long conglomId,
 								int scociItem,
-								GeneratedMethod resultRowAllocator,
+								int resultRowTemplate,
 								int resultSetNumber,
 								GeneratedMethod startKeyGetter,
 								int startSearchOperator,
@@ -1028,7 +1012,7 @@ public interface ResultSetFactory {
 			                    Activation activation,
 								long conglomId,
 								int scociItem,
-								GeneratedMethod resultRowAllocator,
+								int resultRowTemplate,
 								int resultSetNumber,
 								GeneratedMethod startKeyGetter,
 								int startSearchOperator,
@@ -1487,10 +1471,7 @@ public interface ResultSetFactory {
 	 * @param activation 		the activation for this result set,
 	 *		which provides the context for the row allocation operation.
 	 * @param resultSetNumber	The resultSetNumber for the ResultSet
-	 * @param resultRowAllocator a reference to a method in the activation
-	 * 						that creates a holder for the result row of the scan.  May
-	 *						be a partial row.  <verbatim>
-	 *		ExecRow rowAllocator() throws StandardException; </verbatim>
+     * @param resultRowTemplate The saved item for result row template
 	 * @param conglomId 		the conglomerate of the table to be scanned.
 	 * @param tableName			The full name of the table
 	 * @param userSuppliedOptimizerOverrides		Overrides specified by the user on the sql
@@ -1516,7 +1497,7 @@ public interface ResultSetFactory {
 	(
 		Activation 			activation,
 		int 				resultSetNumber,
-		GeneratedMethod 	resultRowAllocator,
+        int                 resultRowTemplate,
 		long 				conglomId,
 		String 				tableName,
 		String 				userSuppliedOptimizerOverrides,
@@ -1532,7 +1513,7 @@ public interface ResultSetFactory {
 
 	/**
 		A Dependent table scan result set forms a result set on a scan
-		of a dependent table for the rows that got materilized 
+		of a dependent table for the rows that got materialized
 		on the scan of its parent table and if the row being deleted
 		on parent table has a reference in the dependent table.
 
@@ -1540,12 +1521,7 @@ public interface ResultSetFactory {
 			which provides the context for the row allocation operation.
 		@param conglomId the conglomerate of the table to be scanned.
 		@param scociItem The saved item for the static conglomerate info.
-		@param resultRowAllocator a reference to a method in the activation
-			that creates a holder for the result row of the scan.  May
-			be a partial row.
-			<verbatim>
-				ExecRow rowAllocator() throws StandardException;
-			</verbatim>
+        @param resultRowTemplate The saved item for result row template.
 		@param resultSetNumber	The resultSetNumber for the ResultSet
 		@param startKeyGetter a reference to a method in the activation
 			that gets the start key indexable row for the scan.  Null
@@ -1584,8 +1560,8 @@ public interface ResultSetFactory {
 		@param optimizerEstimatedRowCount	Estimated total # of rows by
 											optimizer
 		@param optimizerEstimatedCost		Estimated total cost by optimizer
-		@param parentResultSetId  Id to access the materlized temporary result
-                            	  set from the refence stored in the activation.
+		@param parentResultSetId  Id to access the materialized temporary result
+                            	  set from the reference stored in the activation.
 		@param fkIndexConglomId foreign key index conglomerate id.
 		@param fkColArrayItem  saved column array object  that matches the foreign key index
 		                       columns  and the resultset from the parent table.
@@ -1599,7 +1575,7 @@ public interface ResultSetFactory {
 			                        Activation activation,
 									long conglomId,
 									int scociItem,							
-									GeneratedMethod resultRowAllocator,
+									int resultRowTemplate,
 									int resultSetNumber,
 									GeneratedMethod startKeyGetter,
 									int startSearchOperator,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/DataTypeDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/DataTypeDescriptor.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/DataTypeDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/types/DataTypeDescriptor.java Fri Dec  7 13:08:04 2012
@@ -28,9 +28,9 @@ import java.sql.Types;
 import java.text.RuleBasedCollator;
 
 import org.apache.derby.catalog.TypeDescriptor;
-import org.apache.derby.catalog.types.BaseTypeIdImpl;
 import org.apache.derby.catalog.types.RowMultiSetImpl;
 import org.apache.derby.catalog.types.TypeDescriptorImpl;
+import org.apache.derby.catalog.types.UserDefinedTypeIdImpl;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.Property;
 import org.apache.derby.iapi.reference.SQLState;
@@ -1775,8 +1775,21 @@ public final class DataTypeDescriptor im
 		 throws IOException, ClassNotFoundException
 	{
 		typeDescriptor = (TypeDescriptorImpl) in.readObject();
-        
-        typeId = TypeId.getBuiltInTypeId(this.getTypeName());
+
+        // Restore typeId from the type descriptor. User-defined types and
+        // built-in types have different methods for getting the type id.
+        if (typeDescriptor.isUserDefinedType()) {
+            try {
+                typeId = TypeId.getUserDefinedTypeId(
+                    ((UserDefinedTypeIdImpl) typeDescriptor.getTypeId())
+                        .getClassName());
+            } catch (StandardException se) {
+                throw (IOException)
+                        new IOException(se.getMessage()).initCause(se);
+            }
+        } else {
+            typeId = TypeId.getBuiltInTypeId(this.getTypeName());
+        }
         
         collationDerivation = in.readInt();
 	}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseJoinStrategy.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/BaseJoinStrategy.java Fri Dec  7 13:08:04 2012
@@ -79,7 +79,6 @@ abstract class BaseJoinStrategy implemen
 	 * @param innerTable
 	 * @param predList
 	 * @param acbi
-	 * @param resultRowAllocator
 	 * @throws StandardException
 	 */
 	void fillInScanArgs1(
@@ -88,7 +87,7 @@ abstract class BaseJoinStrategy implemen
 								Optimizable innerTable,
 								OptimizablePredicateList predList,
 								ExpressionClassBuilderInterface acbi,
-								MethodBuilder resultRowAllocator
+								int resultRowTemplate
 								)
 					throws StandardException {
 		boolean				   sameStartStopPosition = predList.sameStartStopPosition();
@@ -103,7 +102,7 @@ abstract class BaseJoinStrategy implemen
 		mb.push(conglomNumber);
 		mb.push(acb.addItem(scoci));
 
-		acb.pushMethodReference(mb, resultRowAllocator);
+        mb.push(resultRowTemplate);
 		mb.push(innerTable.getResultSetNumber());
 
 		predList.generateStartKey(acb, mb, innerTable);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/DeleteNode.java Fri Dec  7 13:08:04 2012
@@ -21,36 +21,23 @@
 
 package	org.apache.derby.impl.sql.compile;
 
-import org.apache.derby.iapi.services.context.ContextManager;
-
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.error.StandardException;
 
 import org.apache.derby.iapi.sql.conn.Authorizer;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
-import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.dictionary.GenericDescriptorList;
 import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ColumnDescriptorList;
 import org.apache.derby.iapi.sql.dictionary.TriggerDescriptor;
 
-
-import org.apache.derby.iapi.sql.ResultSet;
 import org.apache.derby.iapi.sql.StatementType;
 
-import org.apache.derby.iapi.sql.compile.CompilerContext;
 import org.apache.derby.iapi.sql.compile.C_NodeTypes;
 import org.apache.derby.iapi.reference.ClassName;
 
-import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
-
-import org.apache.derby.iapi.sql.execute.CursorResultSet;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
-import org.apache.derby.iapi.sql.execute.ExecPreparedStatement;
-import org.apache.derby.iapi.sql.execute.ExecRow;
-
-import org.apache.derby.iapi.sql.Activation;
 
 import org.apache.derby.iapi.services.sanity.SanityManager;
 
@@ -64,8 +51,6 @@ import org.apache.derby.vti.DeferModific
 import org.apache.derby.catalog.UUID;
 import org.apache.derby.iapi.services.io.FormatableBitSet;
 
-import org.apache.derby.impl.sql.compile.ActivationClassBuilder;
-
 import org.apache.derby.impl.sql.execute.FKInfo;
 
 import java.lang.reflect.Modifier;
@@ -78,7 +63,6 @@ import java.util.Iterator;
 import java.util.Properties;
 import org.apache.derby.iapi.sql.compile.NodeFactory;
 import org.apache.derby.iapi.util.ReuseFactory;
-import org.apache.derby.iapi.sql.depend.Dependent;
 import org.apache.derby.iapi.sql.ResultDescription;
 import org.apache.derby.iapi.services.compiler.LocalField;
 
@@ -100,7 +84,6 @@ public class DeleteNode extends DMLModSt
 
 	/* Filled in by bind. */
 	protected boolean				deferred;
-	protected ExecRow				emptyHeapRow;
 	protected FromTable				targetTable;
 	protected FKInfo				fkInfo;
 	protected FormatableBitSet readColsBitSet;
@@ -270,11 +253,6 @@ public class DeleteNode extends DMLModSt
 					readColsBitSet = null;
 				}
 
-				/*
-				** Construct an empty heap row for use in our constant action.
-				*/
-				emptyHeapRow = targetTableDescriptor.getEmptyExecRow();
-
 				/* Generate the RowLocation column */
 				rowLocationNode = (CurrentRowLocationNode) getNodeFactory().getNode(
 										C_NodeTypes.CURRENT_ROW_LOCATION_NODE,
@@ -473,7 +451,6 @@ public class DeleteNode extends DMLModSt
 				  indicesToMaintain,
 				  indexConglomerateNumbers,
 				  indexSCOCIs,
-				  emptyHeapRow,
 				  deferred,
 				  false,
 				  targetTableDescriptor.getUUID(),

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/FromBaseTable.java Fri Dec  7 13:08:04 2012
@@ -3403,7 +3403,7 @@ public class FromBaseTable extends FromT
 
 		acb.pushThisAsActivation(mb);
 		mb.push(getResultSetNumber());
-		resultColumns.generateHolder(acb, mb, referencedCols, (FormatableBitSet) null);
+        mb.push(acb.addItem(resultColumns.buildRowTemplate(referencedCols)));
 		mb.push(cd.getConglomerateNumber());
 		mb.push(tableDescriptor.getName());
 		//User may have supplied optimizer overrides in the sql
@@ -3496,7 +3496,7 @@ public class FromBaseTable extends FromT
      	acb.pushThisAsActivation(mb);
 		mb.push(conglomNumber);
 		mb.push(acb.addItem(scoci));
- 		resultColumns.generateHolder(acb, mb, referencedCols, (FormatableBitSet) null);
+        mb.push(acb.addItem(resultColumns.buildRowTemplate(referencedCols)));
 		mb.push(getResultSetNumber());
 		mb.push(hashKeyItem);
 		mb.push(tableDescriptor.getName());
@@ -3568,11 +3568,9 @@ public class FromBaseTable extends FromT
 										  MethodBuilder mb)
 		throws StandardException
 	{
-        // get a function to allocate scan rows of the right shape and size
-   	    MethodBuilder resultRowAllocator =
-						resultColumns.generateHolderMethod(acb,
-													referencedCols,
-													(FormatableBitSet) null);
+        // Put the result row template in the saved objects.
+        int resultRowTemplate =
+                acb.addItem(resultColumns.buildRowTemplate(referencedCols));
 
 		// pass in the referenced columns on the saved objects
 		// chain
@@ -3623,7 +3621,7 @@ public class FromBaseTable extends FromT
 											nonStoreRestrictionList,
 											acb,
 											bulkFetch,
-											resultRowAllocator,
+											resultRowTemplate,
 											colRefItem,
 											indexColItem,
 											getTrulyTheBestAccessPath().

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/HashJoinStrategy.java Fri Dec  7 13:08:04 2012
@@ -37,10 +37,6 @@ import org.apache.derby.iapi.store.acces
 
 import org.apache.derby.iapi.services.compiler.MethodBuilder;
 
-import org.apache.derby.impl.sql.compile.ExpressionClassBuilder;
-import org.apache.derby.impl.sql.compile.ProjectRestrictNode;
-import org.apache.derby.impl.sql.compile.Predicate;
-
 import org.apache.derby.iapi.error.StandardException;
 
 import org.apache.derby.iapi.reference.SQLState;
@@ -327,7 +323,7 @@ public class HashJoinStrategy extends Ba
 							OptimizablePredicateList nonStoreRestrictionList,
 							ExpressionClassBuilderInterface acbi,
 							int bulkFetch,
-							MethodBuilder resultRowAllocator,
+							int resultRowTemplate,
 							int colRefItem,
 							int indexColItem,
 							int lockMode,
@@ -375,7 +371,7 @@ public class HashJoinStrategy extends Ba
 										innerTable,
 										storeRestrictionList,
 										acb,
-										resultRowAllocator);
+										resultRowTemplate);
 
 		nonStoreRestrictionList.generateQualifiers(acb,	mb, innerTable, true);
 		mb.push(innerTable.initialCapacity());

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/InsertNode.java Fri Dec  7 13:08:04 2012
@@ -21,12 +21,6 @@
 
 package	org.apache.derby.impl.sql.compile;
 
-import org.apache.derby.iapi.services.context.ContextManager;
-
-import org.apache.derby.iapi.services.loader.GeneratedMethod;
-
-import org.apache.derby.iapi.services.compiler.JavaFactory;
-import org.apache.derby.iapi.services.compiler.MethodBuilder;
 
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.error.StandardException;
@@ -35,23 +29,17 @@ import org.apache.derby.iapi.sql.compile
 
 import org.apache.derby.iapi.sql.conn.Authorizer;
 
-import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
-import org.apache.derby.iapi.sql.compile.Visitable;
 import org.apache.derby.iapi.sql.compile.Visitor;
 import org.apache.derby.iapi.sql.compile.CompilerContext;
 
 import org.apache.derby.iapi.reference.ClassName;
 
+import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
+import org.apache.derby.iapi.sql.dictionary.ColumnDescriptorList;
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
-import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptor;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.dictionary.IndexLister;
-import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
-import org.apache.derby.iapi.types.TypeId;
-
-import org.apache.derby.iapi.sql.ResultSet;
-import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.StatementType;
 
 import org.apache.derby.iapi.sql.execute.ConstantAction;
@@ -60,7 +48,6 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.store.access.TransactionController;
 import org.apache.derby.iapi.types.RowLocation;
 
-import org.apache.derby.impl.sql.compile.ActivationClassBuilder;
 import org.apache.derby.iapi.services.compiler.MethodBuilder;
 
 import org.apache.derby.iapi.services.sanity.SanityManager;
@@ -77,6 +64,7 @@ import org.apache.derby.impl.sql.execute
 import java.util.Properties;
 
 import org.apache.derby.iapi.services.io.FormatableBitSet;
+import org.apache.derby.iapi.sql.execute.ExecRowBuilder;
 import org.apache.derby.iapi.util.ReuseFactory;
 
 /**
@@ -889,6 +877,8 @@ public final class InsertNode extends DM
 
         if ( targetProperties.getProperty( key ) == null )
         { targetProperties.put( key, value ); }
+
+        bulkInsert = true;
     }
 
 	/**
@@ -933,7 +923,23 @@ public final class InsertNode extends DM
 			// arg 3 generate code to evaluate CHECK CONSTRAINTS
 			generateCheckConstraints( checkConstraints, acb, mb );
 
-			mb.callMethod(VMOpcode.INVOKEINTERFACE, (String) null, "getInsertResultSet", ClassName.ResultSet, 3);
+            // arg 4 row template used by bulk insert
+            if (bulkInsert) {
+                ColumnDescriptorList cdl =
+                        targetTableDescriptor.getColumnDescriptorList();
+                ExecRowBuilder builder =
+                        new ExecRowBuilder(cdl.size(), false);
+                for (int i = 0; i < cdl.size(); i++) {
+                    ColumnDescriptor cd = (ColumnDescriptor) cdl.get(i);
+                    builder.setColumn(i + 1, cd.getType());
+                }
+                mb.push(acb.addItem(builder));
+            } else {
+                mb.push(-1);
+            }
+
+            mb.callMethod(VMOpcode.INVOKEINTERFACE, (String) null,
+                    "getInsertResultSet", ClassName.ResultSet, 4);
 		}
 		else
 		{

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/NestedLoopJoinStrategy.java Fri Dec  7 13:08:04 2012
@@ -37,8 +37,6 @@ import org.apache.derby.iapi.store.acces
 
 import org.apache.derby.iapi.services.compiler.MethodBuilder;
 
-import org.apache.derby.impl.sql.compile.ExpressionClassBuilder;
-
 import org.apache.derby.iapi.error.StandardException;
 
 import org.apache.derby.iapi.services.sanity.SanityManager;
@@ -203,7 +201,7 @@ public class NestedLoopJoinStrategy exte
 							OptimizablePredicateList nonStoreRestrictionList,
 							ExpressionClassBuilderInterface acbi,
 							int bulkFetch,
-							MethodBuilder resultRowAllocator,
+							int resultRowTemplate,
 							int colRefItem,
 							int indexColItem,
 							int lockMode,
@@ -253,7 +251,7 @@ public class NestedLoopJoinStrategy exte
 										innerTable,
 										storeRestrictionList,
 										acb,
-										resultRowAllocator);
+										resultRowTemplate);
 
 		if (genInListVals)
 			((PredicateList)storeRestrictionList).generateInListValues(acb, mb);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/ResultColumnList.java Fri Dec  7 13:08:04 2012
@@ -42,8 +42,6 @@ import org.apache.derby.iapi.services.lo
 import org.apache.derby.iapi.services.sanity.SanityManager;
 import org.apache.derby.iapi.sql.ResultColumnDescriptor;
 import org.apache.derby.iapi.sql.compile.C_NodeTypes;
-import org.apache.derby.iapi.sql.compile.CompilerContext;
-import org.apache.derby.iapi.sql.compile.Parser;
 import org.apache.derby.iapi.sql.compile.NodeFactory;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
@@ -53,6 +51,7 @@ import org.apache.derby.iapi.sql.diction
 import org.apache.derby.iapi.sql.dictionary.DefaultDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.execute.ExecRow;
+import org.apache.derby.iapi.sql.execute.ExecRowBuilder;
 import org.apache.derby.iapi.store.access.ConglomerateController;
 import org.apache.derby.iapi.store.access.StoreCostController;
 import org.apache.derby.iapi.store.access.TransactionController;
@@ -1511,6 +1510,43 @@ public class ResultColumnList extends Qu
 		return	row;
 	}
 
+    /**
+     * Build an {@code ExecRowBuilder} instance that produces a row of the
+     * same shape as this result column list.
+     *
+     * @param referencedCols a bit map that tells which columns in the
+     * source result set that are used, or {@code null} if all are used
+     * @return an instance that produces rows of the same shape as this
+     * result column list
+     */
+    ExecRowBuilder buildRowTemplate(FormatableBitSet referencedCols)
+            throws StandardException
+    {
+        int columns = (referencedCols == null) ?
+                size() : referencedCols.getNumBitsSet();
+
+        ExecRowBuilder builder = new ExecRowBuilder(columns, indexRow);
+
+        int colNum = (referencedCols == null) ? 0 : referencedCols.anySetBit();
+
+        for (int i = 0; i < size(); i++) {
+            ResultColumn rc = (ResultColumn) elementAt(i);
+
+            if (rc.getExpression() instanceof CurrentRowLocationNode) {
+                builder.setColumn(colNum + 1, newRowLocationTemplate());
+            } else {
+                builder.setColumn(colNum + 1, rc.getType());
+            }
+
+            if (referencedCols == null) {
+                colNum++;
+            } else {
+                colNum = referencedCols.anySetBit(colNum);
+            }
+        }
+
+        return builder;
+    }
 
 	/**
 		Generates a row with the size and shape of the ResultColumnList.
@@ -1630,37 +1666,8 @@ public class ResultColumnList extends Qu
 			 */
 			if (rc.getExpression() instanceof CurrentRowLocationNode)
 			{
-				ConglomerateController cc = null;
-				int savedItem;
-				RowLocation rl;
-				
-				LanguageConnectionContext lcc = getLanguageConnectionContext();
-				DataDictionary dd = lcc.getDataDictionary();
-				
-				int isolationLevel = (dd.getCacheMode() == DataDictionary.DDL_MODE) ? 
-						TransactionController.ISOLATION_READ_COMMITTED : TransactionController.ISOLATION_NOLOCK;
-
-				cc = lcc.getTransactionCompile().openConglomerate(
-						conglomerateId,
-                        false,
-						0,
-						TransactionController.MODE_RECORD,
-						isolationLevel);
-
-				try
-				{
-					rl = cc.newRowLocationTemplate();
-				}
-				finally
-				{
-					if (cc != null)
-					{
-						cc.close();
-					}
-				}
+                int savedItem = acb.addItem(newRowLocationTemplate());
 
-				savedItem = acb.addItem(rl);
-								
 				// get the RowLocation template
 				exprFun.getField(lf); // instance for setColumn
 				exprFun.push(highestColumnNumber + 1); // first arg
@@ -1771,6 +1778,33 @@ public class ResultColumnList extends Qu
 		cb.statementNumHitLimit(1);		// ignore return value
 	}
 
+    /**
+     * Create a row location template of the right type for the source
+     * conglomerate.
+     */
+    private RowLocation newRowLocationTemplate() throws StandardException {
+        LanguageConnectionContext lcc = getLanguageConnectionContext();
+        DataDictionary dd = lcc.getDataDictionary();
+
+        int isolationLevel = (dd.getCacheMode() == DataDictionary.DDL_MODE) ?
+                TransactionController.ISOLATION_READ_COMMITTED :
+                TransactionController.ISOLATION_NOLOCK;
+
+        ConglomerateController cc =
+            lcc.getTransactionCompile().openConglomerate(
+                conglomerateId,
+                false,
+                0,
+                TransactionController.MODE_RECORD,
+                isolationLevel);
+
+        try {
+            return cc.newRowLocationTemplate();
+        } finally {
+            cc.close();
+        }
+    }
+
 	/**
 	 * Make a ResultDescription for use in a ResultSet.
 	 * This is useful when generating/executing a NormalizeResultSet, since

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/UpdateNode.java Fri Dec  7 13:08:04 2012
@@ -84,7 +84,6 @@ public final class UpdateNode extends DM
 	//Note: These are public so they will be visible to
 	//the RepUpdateNode.
 	public int[]				changedColumnIds;
-	public ExecRow				emptyHeapRow;
 	public boolean				deferred;
 	public ValueNode			checkConstraints;
 	public FKInfo				fkInfo;
@@ -486,11 +485,6 @@ public final class UpdateNode extends DM
 
 		if (targetVTI == null)
 		{
-			/*
-			** Construct an empty heap row for use in our constant action.
-			*/
-			emptyHeapRow = targetTableDescriptor.getEmptyExecRow();
-
 			/* Append the list of "after" columns to the list of "before" columns,
 			 * preserving the afterColumns list.  (Necessary for binding
 			 * check constraints.)
@@ -723,7 +717,6 @@ public final class UpdateNode extends DM
 			  indexConglomerateNumbers,
 			  indexSCOCIs,
 			  indexNames,
-			  emptyHeapRow,
 			  deferred,
 			  targetTableDescriptor.getUUID(),
 			  lockMode,

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?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- 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 Fri Dec  7 13:08:04 2012
@@ -85,7 +85,7 @@ class BulkTableScanResultSet extends Tab
 	 */
     BulkTableScanResultSet(long conglomId,
 		StaticCompiledOpenConglomInfo scoci, Activation activation, 
-		GeneratedMethod resultRowAllocator, 
+		int resultRowTemplate,
 		int resultSetNumber,
 		GeneratedMethod startKeyGetter, int startSearchOperator,
 		GeneratedMethod stopKeyGetter, int stopSearchOperator,
@@ -111,7 +111,7 @@ class BulkTableScanResultSet extends Tab
 		super(conglomId,
 			scoci,
 			activation,
-			resultRowAllocator,
+			resultRowTemplate,
 			resultSetNumber,
 			startKeyGetter,
 			startSearchOperator,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteConstantAction.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DeleteConstantAction.java Fri Dec  7 13:08:04 2012
@@ -23,14 +23,10 @@ package org.apache.derby.impl.sql.execut
 
 import org.apache.derby.iapi.services.io.ArrayUtil;
 import org.apache.derby.iapi.services.io.StoredFormatIds;
-import org.apache.derby.iapi.services.io.FormatIdUtil;
 
 import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
 
 import org.apache.derby.iapi.sql.execute.ConstantAction;
-import org.apache.derby.iapi.sql.execute.ExecRow;
-
-import org.apache.derby.iapi.error.StandardException;
 
 import org.apache.derby.iapi.store.access.StaticCompiledOpenConglomInfo;
 
@@ -84,7 +80,6 @@ public class DeleteConstantAction extend
 	 *  @param irgs			Index descriptors
 	 *  @param indexCIDS	Conglomerate IDs of indices
 	 *	@param indexSCOCIs	StaticCompiledOpenConglomInfos for indexes.
-	 *  @param emptyHeapRow	Template for heap row.
 	 *  @param deferred		True means process as a deferred insert.
 	 *	@param targetUUID	UUID of target table
 	 *	@param lockMode		The lock mode to use
@@ -102,7 +97,6 @@ public class DeleteConstantAction extend
 								IndexRowGenerator[]	irgs,
 								long[]				indexCIDS,
 								StaticCompiledOpenConglomInfo[] indexSCOCIs,
-								ExecRow				emptyHeapRow,
 								boolean				deferred,
 								UUID				targetUUID,
 								int					lockMode,
@@ -126,7 +120,6 @@ public class DeleteConstantAction extend
 			   lockMode,
 			   fkInfo,
 			   triggerInfo,
-			   emptyHeapRow,
 			   baseRowReadList,
 			   baseRowReadMap,
 			   streamStorableHeapColIds,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DependentResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DependentResultSet.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DependentResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DependentResultSet.java Fri Dec  7 13:08:04 2012
@@ -117,7 +117,7 @@ class DependentResultSet extends ScanRes
 		long conglomId,
 		StaticCompiledOpenConglomInfo scoci, 
 		Activation activation, 
-		GeneratedMethod resultRowAllocator, 
+		int resultRowTemplate,
 		int resultSetNumber,
 		GeneratedMethod startKeyGetter, int startSearchOperator,
 		GeneratedMethod stopKeyGetter, int stopSearchOperator,
@@ -142,7 +142,7 @@ class DependentResultSet extends ScanRes
 		int rltItem
 		)	throws StandardException
 	{
-		super(activation, resultSetNumber, resultRowAllocator,
+		super(activation, resultSetNumber, resultRowTemplate,
 			  lockMode, tableLocked,
 			  //Because the scan for the tables in this result set are done
 			  //internally for delete cascades, isolation should be set to
@@ -164,7 +164,6 @@ class DependentResultSet extends ScanRes
 
 		if (SanityManager.DEBUG) {
 			SanityManager.ASSERT( activation!=null, "table scan must get activation context");
-			SanityManager.ASSERT( resultRowAllocator!= null, "table scan must get row allocator");
 
             // This ResultSet doesn't use start or stop keys, so expect them
             // to be null.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DistinctScanResultSet.java Fri Dec  7 13:08:04 2012
@@ -51,7 +51,7 @@ class DistinctScanResultSet extends Hash
     //
     DistinctScanResultSet(long conglomId, 
 		StaticCompiledOpenConglomInfo scoci, Activation activation, 
-		GeneratedMethod resultRowAllocator, 
+		int resultRowTemplate,
 		int resultSetNumber,
 		int hashKeyItem,
 		String tableName,
@@ -66,7 +66,7 @@ class DistinctScanResultSet extends Hash
 		double optimizerEstimatedCost)
 			throws StandardException
     {
-		super(conglomId, scoci, activation, resultRowAllocator, resultSetNumber,
+		super(conglomId, scoci, activation, resultRowTemplate, resultSetNumber,
 			  (GeneratedMethod) null, // startKeyGetter
 			  0,					  // startSearchOperator
 			  (GeneratedMethod) null, // stopKeyGetter

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericConstantActionFactory.java Fri Dec  7 13:08:04 2012
@@ -31,21 +31,15 @@ import org.apache.derby.iapi.sql.conn.La
 import org.apache.derby.iapi.sql.ResultDescription;
 
 import org.apache.derby.iapi.sql.execute.ConstantAction;
-import org.apache.derby.iapi.sql.execute.ExecIndexRow;
 import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptorList;
-import org.apache.derby.iapi.sql.dictionary.GenericDescriptorList;
 import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 
-import org.apache.derby.iapi.sql.execute.ExecRow;
-
 import org.apache.derby.iapi.sql.depend.ProviderInfo;
 
 import org.apache.derby.iapi.store.access.StaticCompiledOpenConglomInfo;
 
-import org.apache.derby.iapi.services.sanity.SanityManager;
-
 import org.apache.derby.iapi.types.RowLocation;
 import org.apache.derby.iapi.types.DataTypeDescriptor;
 
@@ -449,7 +443,6 @@ public class GenericConstantActionFactor
 	 *  @param irgs					Index descriptors
 	 *  @param indexCIDS			Conglomerate IDs of indices
 	 *	@param indexSCOCIs	StaticCompiledOpenConglomInfos for indexes.
-	 *  @param emptyHeapRow			Template for heap row.
 	 *	@param deferred				True means deferred delete
 	 *  @param tableIsPublished		true if table is published
 	 *  @param tableID				table id
@@ -485,7 +478,6 @@ public class GenericConstantActionFactor
 								IndexRowGenerator[]	irgs,
 								long[]				indexCIDS,
 								StaticCompiledOpenConglomInfo[] indexSCOCIs,
-								ExecRow				emptyHeapRow,
 								boolean				deferred,
 								boolean				tableIsPublished,
 								UUID				tableID,
@@ -516,7 +508,6 @@ public class GenericConstantActionFactor
 										irgs,
 										indexCIDS,
 										indexSCOCIs,
-										emptyHeapRow,
 										deferred,
 										tableID,
 										lockMode,
@@ -864,7 +855,6 @@ public class GenericConstantActionFactor
 	 *  @param irgs					Index descriptors
 	 *  @param indexCIDS			Conglomerate IDs of indices
 	 *	@param indexSCOCIs	StaticCompiledOpenConglomInfos for indexes.
-	 *  @param emptyHeapRow			Template for heap row.
 	 *	@param deferred				True means deferred update
 	 *	@param targetUUID			UUID of target table
 	 *	@param lockMode				The lock mode to use
@@ -893,7 +883,6 @@ public class GenericConstantActionFactor
 								long[]				indexCIDS,
 								StaticCompiledOpenConglomInfo[] indexSCOCIs,
 								String[]			indexNames,	
-								ExecRow				emptyHeapRow,
 								boolean				deferred,
 								UUID				targetUUID,
 								int					lockMode,
@@ -919,7 +908,6 @@ public class GenericConstantActionFactor
 										indexCIDS,
 										indexSCOCIs,
 										indexNames,
-										emptyHeapRow,
 										deferred,
 										targetUUID,
 										lockMode,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/GenericResultSetFactory.java Fri Dec  7 13:08:04 2012
@@ -21,7 +21,6 @@
 
 package org.apache.derby.impl.sql.execute;
 
-import org.apache.derby.catalog.TypeDescriptor;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.services.loader.GeneratedMethod;
 import org.apache.derby.iapi.services.sanity.SanityManager;
@@ -68,12 +67,13 @@ public class GenericResultSetFactory imp
 		@exception StandardException thrown on error
 	 */
 	public ResultSet getInsertResultSet(NoPutResultSet source, GeneratedMethod generationClauses,
-										GeneratedMethod checkGM)
+                                        GeneratedMethod checkGM, int fullTemplate)
 		throws StandardException
 	{
 		Activation activation = source.getActivation();
 		getAuthorizer(activation).authorize(activation, Authorizer.SQL_WRITE_OP);
-		return new InsertResultSet(source, generationClauses, checkGM, activation );
+        return new InsertResultSet(
+                source, generationClauses, checkGM, fullTemplate, activation);
 	}
 
 	/**
@@ -483,7 +483,7 @@ public class GenericResultSetFactory imp
                         			Activation activation,
 									long conglomId,
 									int scociItem,
-									GeneratedMethod resultRowAllocator,
+									int resultRowTemplate,
 									int resultSetNumber,
 									GeneratedMethod startKeyGetter,
 									int startSearchOperator,
@@ -517,7 +517,7 @@ public class GenericResultSetFactory imp
 								conglomId,
 								scoci,
 								activation,
-								resultRowAllocator,
+								resultRowTemplate,
 								resultSetNumber,
 								startKeyGetter,
 								startSearchOperator,
@@ -553,7 +553,7 @@ public class GenericResultSetFactory imp
                          			Activation activation,
 									long conglomId,
 									int scociItem,
-									GeneratedMethod resultRowAllocator,
+									int resultRowTemplate,
 									int resultSetNumber,
 									int hashKeyColumn,
 									String tableName,
@@ -574,7 +574,7 @@ public class GenericResultSetFactory imp
 								conglomId,
 								scoci,
 								activation,
-								resultRowAllocator,
+								resultRowTemplate,
 								resultSetNumber,
 								hashKeyColumn,
 								tableName,
@@ -598,7 +598,7 @@ public class GenericResultSetFactory imp
                         			Activation activation,
 									long conglomId,
 									int scociItem,
-									GeneratedMethod resultRowAllocator,
+									int resultRowTemplate,
 									int resultSetNumber,
 									GeneratedMethod startKeyGetter,
 									int startSearchOperator,
@@ -627,7 +627,7 @@ public class GenericResultSetFactory imp
 								conglomId,
 								scoci,
 								activation,
-								resultRowAllocator,
+								resultRowTemplate,
 								resultSetNumber,
 								startKeyGetter,
 								startSearchOperator,
@@ -660,7 +660,7 @@ public class GenericResultSetFactory imp
                        			    Activation activation,
 									long conglomId,
 									int scociItem,
-									GeneratedMethod resultRowAllocator,
+									int resultRowTemplate,
 									int resultSetNumber,
 									GeneratedMethod startKeyGetter,
 									int startSearchOperator,
@@ -698,7 +698,7 @@ public class GenericResultSetFactory imp
 								conglomId,
 								scoci,
 								activation,
-								resultRowAllocator,
+								resultRowTemplate,
 								resultSetNumber,
 								startKeyGetter,
 								startSearchOperator,
@@ -739,7 +739,7 @@ public class GenericResultSetFactory imp
 									Activation activation,
 									long conglomId,
 									int scociItem,
-									GeneratedMethod resultRowAllocator,
+									int resultRowTemplate,
 									int resultSetNumber,
 									GeneratedMethod startKeyGetter,
 									int startSearchOperator,
@@ -771,7 +771,7 @@ public class GenericResultSetFactory imp
 								conglomId,
 								scoci,
 								activation,
-								resultRowAllocator,
+								resultRowTemplate,
 								resultSetNumber,
 								startKeyGetter,
 								startSearchOperator,
@@ -1145,10 +1145,7 @@ public class GenericResultSetFactory imp
 	 * @param activation 		the activation for this result set,
 	 *		which provides the context for the row allocation operation.
 	 * @param resultSetNumber	The resultSetNumber for the ResultSet
-	 * @param resultRowAllocator a reference to a method in the activation
-	 * 						that creates a holder for the result row of the scan.  May
-	 *						be a partial row.  <verbatim>
-	 *		ExecRow rowAllocator() throws StandardException; </verbatim>
+     * @param resultRowTemplate The saved item for result row template
 	 * @param conglomId 		the conglomerate of the table to be scanned.
 	 * @param tableName			The full name of the table
 	 * @param userSuppliedOptimizerOverrides		Overrides specified by the user on the sql
@@ -1174,7 +1171,7 @@ public class GenericResultSetFactory imp
 	(
 		Activation 			activation,
 		int 				resultSetNumber,
-		GeneratedMethod 	resultRowAllocator,
+        int                 resultRowTemplate,
 		long 				conglomId,
 		String 				tableName,
 		String 				userSuppliedOptimizerOverrides,
@@ -1190,7 +1187,7 @@ public class GenericResultSetFactory imp
 		return new LastIndexKeyResultSet(
 					activation,
 					resultSetNumber,
-					resultRowAllocator,
+					resultRowTemplate,
 					conglomId,
 					tableName,
 					userSuppliedOptimizerOverrides,
@@ -1214,7 +1211,7 @@ public class GenericResultSetFactory imp
 			                        Activation activation,
 									long conglomId,
 									int scociItem,
-									GeneratedMethod resultRowAllocator,
+									int resultRowTemplate,
 									int resultSetNumber,
 									GeneratedMethod startKeyGetter,
 									int startSearchOperator,
@@ -1247,7 +1244,7 @@ public class GenericResultSetFactory imp
 								conglomId,
 								scoci,
 								activation,
-								resultRowAllocator,
+								resultRowTemplate,
 								resultSetNumber,
 								startKeyGetter,
 								startSearchOperator,

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?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- 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 Fri Dec  7 13:08:04 2012
@@ -112,7 +112,7 @@ public class HashScanResultSet extends S
     //
     HashScanResultSet(long conglomId,
 		StaticCompiledOpenConglomInfo scoci, Activation activation, 
-		GeneratedMethod resultRowAllocator, 
+		int resultRowTemplate,
 		int resultSetNumber,
 		GeneratedMethod startKeyGetter, int startSearchOperator,
 		GeneratedMethod stopKeyGetter, int stopSearchOperator,
@@ -139,7 +139,7 @@ public class HashScanResultSet extends S
     {
 		super(activation,
 				resultSetNumber,
-				resultRowAllocator,
+				resultRowTemplate,
 				lockMode, tableLocked, isolationLevel,
                 colRefItem,
 				optimizerEstimatedRowCount,
@@ -149,7 +149,6 @@ public class HashScanResultSet extends S
 
 		if (SanityManager.DEBUG) {
 			SanityManager.ASSERT( activation!=null, "hash scan must get activation context");
-			SanityManager.ASSERT( resultRowAllocator!= null, "hash scan must get row allocator");
 			if (sameStartStopPosition)
 			{
 				SanityManager.ASSERT(stopKeyGetter == null,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertConstantAction.java Fri Dec  7 13:08:04 2012
@@ -150,7 +150,6 @@ public class InsertConstantAction extend
 			  lockMode,
 			  fkInfo,	
 			  triggerInfo,
-			  (ExecRow)null, // never need to pass in a heap row
 			  null,
 			  null,
 			  streamStorableHeapColIds,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java Fri Dec  7 13:08:04 2012
@@ -55,6 +55,7 @@ import org.apache.derby.iapi.sql.diction
 import org.apache.derby.iapi.sql.execute.CursorResultSet;
 import org.apache.derby.iapi.sql.execute.ExecIndexRow;
 import org.apache.derby.iapi.sql.execute.ExecRow;
+import org.apache.derby.iapi.sql.execute.ExecRowBuilder;
 import org.apache.derby.iapi.sql.execute.NoPutResultSet;
 import org.apache.derby.iapi.sql.execute.RowChanger;
 import org.apache.derby.iapi.sql.execute.TargetResultSet;
@@ -128,7 +129,7 @@ class InsertResultSet extends DMLWriteRe
 	protected TableDescriptor			td;
 		
 	private ExecIndexRow[]			indexRows;
-	private ExecRow					fullTemplate;
+    private final int               fullTemplateId;
 	private	long[]					sortIds;
 	private RowLocationRetRowSource[]
                                     rowSources;
@@ -319,6 +320,7 @@ class InsertResultSet extends DMLWriteRe
     InsertResultSet(NoPutResultSet source, 
 						   GeneratedMethod generationClauses,
 						   GeneratedMethod checkGM,
+                           int fullTemplate,
 						   Activation activation)
 		throws StandardException
     {
@@ -327,6 +329,7 @@ class InsertResultSet extends DMLWriteRe
 		constants = (InsertConstantAction) constantAction;
         this.generationClauses = generationClauses;
 		this.checkGM = checkGM;
+        this.fullTemplateId = fullTemplate;
 		heapConglom = constants.conglomId;
 
         tc = activation.getTransactionController();
@@ -452,7 +455,14 @@ class InsertResultSet extends DMLWriteRe
 		{
 			// Notify the source that we are the target
 			sourceResultSet.setTargetResultSet(this);
-			long baseTableConglom = bulkInsertCore(lcc, heapConglom);
+
+            ExecRow fullTemplate =
+                ((ExecRowBuilder) activation.getPreparedStatement().
+                    getSavedObject(fullTemplateId)).build(
+                        activation.getExecutionFactory());
+
+            long baseTableConglom =
+                    bulkInsertCore(lcc, fullTemplate, heapConglom);
 
 			if (hasBeforeStatementTrigger)
 			{	
@@ -488,7 +498,7 @@ class InsertResultSet extends DMLWriteRe
 				}
 			}
 			
-			bulkValidateForeignKeys(tc, lcc.getContextManager());
+            bulkValidateForeignKeys(tc, lcc.getContextManager(), fullTemplate);
 	
 			// if we have an AFTER trigger, let 'er rip
 			if ((triggerInfo != null) && 
@@ -1252,10 +1262,10 @@ class InsertResultSet extends DMLWriteRe
 
 	// Do the work for a bulk insert
 	private long bulkInsertCore(LanguageConnectionContext lcc,
+                                ExecRow fullTemplate,
 								long oldHeapConglom)
 		throws StandardException
 	{
-		fullTemplate = constants.getEmptyHeapRow(lcc);
 		bulkHeapCC = tc.openCompiledConglomerate(
                                 false,
                                 TransactionController.OPENMODE_FORUPDATE,
@@ -1407,7 +1417,8 @@ class InsertResultSet extends DMLWriteRe
 	/**
 	** Bulk Referential Integrity Checker
 	*/
-	private void bulkValidateForeignKeys(TransactionController tc, ContextManager cm)
+    private void bulkValidateForeignKeys(
+            TransactionController tc, ContextManager cm, ExecRow fullTemplate)
 		throws StandardException
 	{
 		FKInfo 			fkInfo;
@@ -1505,7 +1516,7 @@ class InsertResultSet extends DMLWriteRe
 					}
 					bulkValidateForeignKeysCore(
 							tc, cm, fkInfoArray[i], fkConglom, pkConglom, 
-							fkInfo.fkConstraintNames[index]);
+							fkInfo.fkConstraintNames[index], fullTemplate);
 				}
 			}
 			else
@@ -1525,7 +1536,8 @@ class InsertResultSet extends DMLWriteRe
 										new Long(fkInfo.fkConglomNumbers[0]));
 				bulkValidateForeignKeysCore(
 						tc, cm, fkInfoArray[i], fkConglom.longValue(),
-						fkInfo.refConglomNumber, fkInfo.fkConstraintNames[0]);
+						fkInfo.refConglomNumber, fkInfo.fkConstraintNames[0],
+                        fullTemplate);
 			}
 		}
 	}
@@ -1533,7 +1545,7 @@ class InsertResultSet extends DMLWriteRe
 	private void bulkValidateForeignKeysCore(
 						TransactionController tc, ContextManager cm, 
 						FKInfo fkInfo, long fkConglom, long pkConglom,
-						String fkConstraintName)
+                        String fkConstraintName, ExecRow fullTemplate)
 		throws StandardException
 	{
 		ExecRow 		            template;
@@ -2429,7 +2441,7 @@ class InsertResultSet extends DMLWriteRe
                     conglomId,
                     tc.getStaticCompiledConglomInfo(conglomId),
                     activation,
-                    new MyRowAllocator(fullTemplate),	// result row allocator
+                    fullTemplateId,
                     0,						// result set number
                     (GeneratedMethod)null, 	// start key getter
                     0, 						// start search operator
@@ -2477,21 +2489,4 @@ class InsertResultSet extends DMLWriteRe
 		sourceResultSet.finish();
 		super.finish();
 	}
-
-
-	// inner class to be our row template constructor
-	class MyRowAllocator implements GeneratedMethod
-	{
-		private ExecRow row;
-		MyRowAllocator(ExecRow row)
-		{
-			this.row = row;
-		}
-
-		public Object invoke(Object ref)
-		{
-			return row.getClone();
-		}
-	}
 }
-

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?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- 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 Fri Dec  7 13:08:04 2012
@@ -22,9 +22,9 @@
 package org.apache.derby.impl.sql.execute;
 
 import org.apache.derby.iapi.error.StandardException;
-import org.apache.derby.iapi.services.loader.GeneratedMethod;
 import org.apache.derby.iapi.services.sanity.SanityManager;
 import org.apache.derby.iapi.sql.Activation;
+import org.apache.derby.iapi.sql.execute.CursorResultSet;
 import org.apache.derby.iapi.sql.execute.ExecRow;
 import org.apache.derby.iapi.sql.execute.NoPutResultSet;
 import org.apache.derby.iapi.store.access.Qualifier;
@@ -60,10 +60,7 @@ class LastIndexKeyResultSet extends Scan
 	 * @param activation 		the activation for this result set,
 	 *		which provides the context for the row allocation operation.
 	 * @param resultSetNumber	The resultSetNumber for the ResultSet
-	 * @param resultRowAllocator a reference to a method in the activation
-	 * 						that creates a holder for the result row of the scan.  May
-	 *						be a partial row.  <verbatim>
-	 *		ExecRow rowAllocator() throws StandardException; </verbatim>
+     * @param resultRowTemplate The saved item for result row template
 	 * @param conglomId 		the conglomerate of the table to be scanned.
 	 * @param tableName			The full name of the table
 	 * @param userSuppliedOptimizerOverrides		Overrides specified by the user on the sql
@@ -87,7 +84,7 @@ class LastIndexKeyResultSet extends Scan
 	(
 		Activation activation, 
 		int	resultSetNumber,
-		GeneratedMethod resultRowAllocator, 
+		int resultRowTemplate,
 		long conglomId, 
 		String tableName,
 		String userSuppliedOptimizerOverrides,
@@ -102,7 +99,7 @@ class LastIndexKeyResultSet extends Scan
 	{
 		super(activation,
 				resultSetNumber,
-				resultRowAllocator,
+				resultRowTemplate,
 				lockMode, tableLocked, isolationLevel,
                 colRefItem,
 				optimizerEstimatedRowCount,
@@ -113,7 +110,6 @@ class LastIndexKeyResultSet extends Scan
 		if (SanityManager.DEBUG) 
 		{
 			SanityManager.ASSERT( activation!=null, "this scan must get activation context");
-			SanityManager.ASSERT( resultRowAllocator!= null, "this scan must get row allocator");
 		}
 
 		this.tableName = tableName;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MultiProbeTableScanResultSet.java Fri Dec  7 13:08:04 2012
@@ -105,7 +105,7 @@ class MultiProbeTableScanResultSet exten
      */
     MultiProbeTableScanResultSet(long conglomId,
         StaticCompiledOpenConglomInfo scoci, Activation activation, 
-        GeneratedMethod resultRowAllocator, 
+        int resultRowTemplate,
         int resultSetNumber,
         GeneratedMethod startKeyGetter, int startSearchOperator,
         GeneratedMethod stopKeyGetter, int stopSearchOperator,
@@ -135,7 +135,7 @@ class MultiProbeTableScanResultSet exten
         super(conglomId,
             scoci,
             activation,
-            resultRowAllocator,
+            resultRowTemplate,
             resultSetNumber,
             startKeyGetter,
             startSearchOperator,

Modified: 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?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/ScanResultSet.java Fri Dec  7 13:08:04 2012
@@ -21,10 +21,11 @@ package org.apache.derby.impl.sql.execut
 
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.services.io.FormatableBitSet;
-import org.apache.derby.iapi.services.loader.GeneratedMethod;
 import org.apache.derby.iapi.services.sanity.SanityManager;
 import org.apache.derby.iapi.sql.Activation;
+import org.apache.derby.iapi.sql.execute.ExecPreparedStatement;
 import org.apache.derby.iapi.sql.execute.ExecRow;
+import org.apache.derby.iapi.sql.execute.ExecRowBuilder;
 import org.apache.derby.iapi.sql.execute.ExecutionContext;
 import org.apache.derby.iapi.store.access.TransactionController;
 
@@ -71,6 +72,9 @@ abstract class ScanResultSet extends NoP
     /** The scan isolation level. */
     int isolationLevel;
 
+    /** Object used to create and reset the candidate row. */
+    final ExecRowBuilder resultRowBuilder;
+
     /** The candidate row, matches the shape of the rows in
      * the underlying object to be scanned.
      */
@@ -88,7 +92,7 @@ abstract class ScanResultSet extends NoP
      *
      * @param activation the activation
      * @param resultSetNumber number of the result set (unique within statement)
-     * @param resultRowAllocator method which generates rows
+     * @param resultRowTemplate identifier of saved object for row template
      * @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
@@ -98,7 +102,7 @@ abstract class ScanResultSet extends NoP
      * @param optimizerEstimatedCost estimated cost
      */
     ScanResultSet(Activation activation, int resultSetNumber,
-                  GeneratedMethod resultRowAllocator,
+                  int resultRowTemplate,
                   int lockMode, boolean tableLocked, int isolationLevel,
                   int colRefItem,
                   double optimizerEstimatedRowCount,
@@ -121,12 +125,15 @@ abstract class ScanResultSet extends NoP
         this.isolationLevel =
             translateLanguageIsolationLevel(isolationLevel);
 
-        /* Only call row allocators once */
-        candidate = (ExecRow) resultRowAllocator.invoke(activation);
-        
+        ExecPreparedStatement ps = activation.getPreparedStatement();
+
+        // Create a candidate row.
+        resultRowBuilder =
+                (ExecRowBuilder) ps.getSavedObject(resultRowTemplate);
+        candidate = resultRowBuilder.build(activation.getExecutionFactory());
+
         this.accessedCols = colRefItem != -1 ?
-            (FormatableBitSet)(activation.getPreparedStatement().
-                getSavedObject(colRefItem)) : null;      
+            (FormatableBitSet) ps.getSavedObject(colRefItem) : null;
     }
 
     /**

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?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- 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 Fri Dec  7 13:08:04 2012
@@ -67,7 +67,6 @@ class TableScanResultSet extends ScanRes
 	protected long conglomId;
     protected DynamicCompiledOpenConglomInfo dcoci;
     protected StaticCompiledOpenConglomInfo scoci;
-	protected GeneratedMethod resultRowAllocator;
 	protected GeneratedMethod startKeyGetter;
 	protected int startSearchOperator;
 	protected GeneratedMethod stopKeyGetter;
@@ -126,7 +125,7 @@ class TableScanResultSet extends ScanRes
     TableScanResultSet(long conglomId,
 		StaticCompiledOpenConglomInfo scoci, 
 		Activation activation, 
-		GeneratedMethod resultRowAllocator, 
+		int resultRowTemplate,
 		int resultSetNumber,
 		GeneratedMethod startKeyGetter, int startSearchOperator,
 		GeneratedMethod stopKeyGetter, int stopSearchOperator,
@@ -150,7 +149,7 @@ class TableScanResultSet extends ScanRes
     {
 		super(activation,
 				resultSetNumber,
-				resultRowAllocator,
+				resultRowTemplate,
 				lockMode, tableLocked, isolationLevel,
                 colRefItem,
 				optimizerEstimatedRowCount,
@@ -167,7 +166,6 @@ class TableScanResultSet extends ScanRes
 
 		if (SanityManager.DEBUG) {
 			SanityManager.ASSERT( activation!=null, "table scan must get activation context");
-			SanityManager.ASSERT( resultRowAllocator!= null, "table scan must get row allocator");
 			if (sameStartStopPosition)
 			{
 				SanityManager.ASSERT(stopKeyGetter == null,
@@ -175,8 +173,6 @@ class TableScanResultSet extends ScanRes
 			}
 		}
 
-        this.resultRowAllocator = resultRowAllocator;
-
 		this.startKeyGetter = startKeyGetter;
 		this.startSearchOperator = startSearchOperator;
 		this.stopKeyGetter = stopKeyGetter;
@@ -734,8 +730,6 @@ class TableScanResultSet extends ScanRes
 	 */
 	public ExecRow getCurrentRow() throws StandardException 
 	{
-	    ExecRow result = null;
-
 		if (SanityManager.DEBUG)
 			SanityManager.ASSERT(isOpen, "TSRS expected to be open");
 
@@ -766,13 +760,12 @@ class TableScanResultSet extends ScanRes
 			}
 		}
 
-		result = (ExecRow) resultRowAllocator.invoke(activation);
-		currentRow = 
-            getCompactRow(result, accessedCols, isKeyed);
+        resultRowBuilder.reset(candidate);
+        currentRow = getCompactRow(candidate, accessedCols, isKeyed);
 
         try
         {
-            scanController.fetchWithoutQualify(result.getRowArray());
+            scanController.fetchWithoutQualify(candidate.getRowArray());
         }
         catch (StandardException se)
         {
@@ -789,7 +782,7 @@ class TableScanResultSet extends ScanRes
             }
         }
 
-		setCurrentRow(result);
+        setCurrentRow(candidate);
 	    return currentRow;
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdatableVTIConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdatableVTIConstantAction.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdatableVTIConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdatableVTIConstantAction.java Fri Dec  7 13:08:04 2012
@@ -21,35 +21,7 @@
 
 package org.apache.derby.impl.sql.execute;
 
-import org.apache.derby.iapi.services.stream.HeaderPrintWriter;
-
-
-import org.apache.derby.iapi.services.io.ArrayUtil;
 import org.apache.derby.iapi.services.io.StoredFormatIds;
-import org.apache.derby.iapi.services.io.FormatIdUtil;
-
-import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
-
-import org.apache.derby.iapi.sql.execute.ConstantAction;
-import org.apache.derby.iapi.sql.execute.ExecRow;
-
-import org.apache.derby.iapi.sql.Activation;
-
-import org.apache.derby.iapi.error.StandardException;
-
-import org.apache.derby.iapi.store.access.StaticCompiledOpenConglomInfo;
-
-import org.apache.derby.catalog.UUID;
-
-import java.io.ObjectOutput;
-import java.io.ObjectInput;
-import java.io.IOException;
-import java.io.Serializable;
-
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-
-import java.util.Properties;
 
 /**
  *	This class  describes compiled constants that are passed into
@@ -105,7 +77,6 @@ public class UpdatableVTIConstantAction 
 			  0,
 			  null,	
 			  null,
-			  (ExecRow)null, // never need to pass in a heap row
 			  null,
 			  null,
 			  null,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateConstantAction.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/UpdateConstantAction.java Fri Dec  7 13:08:04 2012
@@ -21,21 +21,11 @@
 
 package org.apache.derby.impl.sql.execute;
 
-import org.apache.derby.iapi.services.stream.HeaderPrintWriter;
-
 import org.apache.derby.iapi.services.io.ArrayUtil;
 import org.apache.derby.iapi.services.io.StoredFormatIds;
-import org.apache.derby.iapi.services.io.FormatIdUtil;
 
 import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
 
-import org.apache.derby.iapi.sql.execute.ConstantAction;
-import org.apache.derby.iapi.sql.execute.ExecRow;
-
-import org.apache.derby.iapi.sql.Activation;
-
-import org.apache.derby.iapi.error.StandardException;
-
 import org.apache.derby.iapi.store.access.StaticCompiledOpenConglomInfo;
 
 import org.apache.derby.catalog.UUID;
@@ -92,7 +82,6 @@ public class UpdateConstantAction extend
 	 *  @param indexCIDS	Conglomerate IDs of indices
 	 *	@param indexSCOCIs	StaticCompiledOpenConglomInfos for indexes.
 	 *  @param indexNames	Names of indices on this table for error reporting.
-	 *  @param emptyHeapRow	Template for heap row.
 	 *  @param deferred		True means process as a deferred update.
 	 *	@param targetUUID	UUID of target table
 	 *	@param lockMode		The lock mode to use
@@ -116,7 +105,6 @@ public class UpdateConstantAction extend
 								long[]				indexCIDS,
 								StaticCompiledOpenConglomInfo[] indexSCOCIs,
 								String[]			indexNames,
-								ExecRow				emptyHeapRow,
 								boolean				deferred,
 								UUID				targetUUID,
 								int					lockMode,
@@ -143,7 +131,6 @@ public class UpdateConstantAction extend
 			lockMode,
 			fkInfo,
 			triggerInfo,
-			emptyHeapRow,
 			baseRowReadList,
 			baseRowReadMap,
 			streamStorableHeapColIds,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WriteCursorConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WriteCursorConstantAction.java?rev=1418297&r1=1418296&r2=1418297&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WriteCursorConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/WriteCursorConstantAction.java Fri Dec  7 13:08:04 2012
@@ -33,12 +33,8 @@ import org.apache.derby.iapi.services.io
 import org.apache.derby.iapi.services.io.FormatableBitSet;
 import org.apache.derby.iapi.services.sanity.SanityManager;
 import org.apache.derby.iapi.sql.Activation;
-import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
-import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
-import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
-import org.apache.derby.iapi.sql.execute.ExecRow;
 import org.apache.derby.iapi.store.access.StaticCompiledOpenConglomInfo;
 
 
@@ -82,7 +78,6 @@ abstract	class WriteCursorConstantAction
 	private	FKInfo[]					fkInfo;
 	private TriggerInfo					triggerInfo;
 
-	private ExecRow						emptyHeapRow;
 	private FormatableBitSet baseRowReadList;
 	private int[] baseRowReadMap;
 	private int[] streamStorableHeapColIds;
@@ -112,7 +107,6 @@ abstract	class WriteCursorConstantAction
 	 *	@param lockMode		The lock mode to use on the target table
 	 *	@param fkInfo	Structure containing foreign key info, if any (may be null)
 	 *	@param triggerInfo	Structure containing trigger info, if any (may be null)
-	 *  @param emptyHeapRow	an empty heap row
 	 *  @param baseRowReadMap	BaseRowReadMap[heapColId]->ReadRowColumnId. (0 based)
      *  @param streamStorableHeapColIds Null for non rep. (0 based)
 	 *  @param singleRowSource		Whether or not source is a single row source
@@ -130,7 +124,6 @@ abstract	class WriteCursorConstantAction
 								int					lockMode,
 								FKInfo[]			fkInfo,
 								TriggerInfo			triggerInfo,
-								ExecRow				emptyHeapRow,
 								FormatableBitSet				baseRowReadList,
 								int[]               baseRowReadMap,
 								int[]               streamStorableHeapColIds,
@@ -147,7 +140,6 @@ abstract	class WriteCursorConstantAction
 		this.targetProperties = targetProperties;
 		this.targetUUID = targetUUID;
 		this.lockMode = lockMode;
-		this.emptyHeapRow = emptyHeapRow;
 		this.fkInfo = fkInfo;
 		this.triggerInfo = triggerInfo;
 		this.baseRowReadList = baseRowReadList;
@@ -299,33 +291,6 @@ abstract	class WriteCursorConstantAction
 	 * @return the conglomerate id.
 	 */
 	public long getConglomerateId() { return conglomId; }
-		
-	/**
-	 *	Get emptyHeapRow
-	 *
-	 * @param lcc	The LanguageConnectionContext to use.
-	 *
-	 * @return	an empty base table row for the table being updated.
-	 *
-	 * @exception StandardException on error
-	 */
-	public ExecRow getEmptyHeapRow(LanguageConnectionContext lcc) throws StandardException
-	{
-		DataDictionary dd;
-		TableDescriptor td;
-
-		if (emptyHeapRow == null)
-		{
-
-			dd = lcc.getDataDictionary();
-	
-			td = dd.getTableDescriptor(targetUUID);
-	
-			emptyHeapRow = td.getEmptyExecRow();
-		}
-
-		return emptyHeapRow.getClone();
-	}
 
 	/**
 	 * Get the targetProperties from the constant action.